You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2016/08/13 00:12:29 UTC

[1/3] hive git commit: HIVE-14444 Upgrade qtest execution framework to junit4 - migrate most of them (Zoltan Haindrich via Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master bfdc74ff2 -> 1f7e10c03


http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/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
new file mode 100644
index 0000000..8620cde
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CorePerfCliDriver.java
@@ -0,0 +1,138 @@
+package org.apache.hadoop.hive.cli.control;
+/**
+ * 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.
+ */
+
+
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.hive.ql.QTestUtil;
+import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
+import org.junit.After;
+import org.junit.AfterClass;
+/**
+ This is the TestPerformance Cli Driver for integrating performance regression tests
+ as part of the Hive Unit tests.
+ Currently this includes support for :
+ 1. Running explain plans for TPCDS workload (non-partitioned dataset)  on 30TB scaleset.
+ TODO :
+ 1. Support for partitioned data set
+ 2. Use HBase Metastore instead of Derby
+
+This suite differs from TestCliDriver w.r.t the fact that we modify the underlying metastore
+database to reflect the dataset before running the queries.
+*/
+public class CorePerfCliDriver extends CliAdapter{
+
+  private static QTestUtil qt;
+
+  public CorePerfCliDriver(AbstractCliConfig testCliConfig) {
+    super(testCliConfig);
+  }
+
+  @Override
+  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();
+      qt = new QTestUtil(cliConfig.getResultsDir(), cliConfig.getLogDir(), miniMR, hiveConfDir,
+          hadoopVer, initScript,
+          cleanupScript, false, false);
+
+      // do a one time initialization
+      qt.cleanUp();
+      qt.createSources();
+      // Manually modify the underlying metastore db to reflect statistics corresponding to
+      // the 30TB TPCDS scale set. This way the optimizer will generate plans for a 30 TB set.
+      QTestUtil.setupMetaStoreTableColumnStatsFor30TBTPCDSWorkload(qt.getConf());
+    } 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);
+    }
+  }
+
+  @Override
+  @AfterClass
+  public void shutdown() throws Exception {
+    qt.shutdown();
+  }
+
+  @Override
+  public void setUp() {
+  }
+
+  @Override
+  @After
+  public void tearDown() {
+    try {
+      qt.clearPostTestEffects();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in tearDown");
+    }
+  }
+
+  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 {
+    long startTime = System.currentTimeMillis();
+    try {
+      System.err.println("Begin query: " + fname);
+
+      qt.addFile(fpath);
+
+      if (qt.shouldBeSkipped(fname)) {
+        return;
+      }
+
+      qt.cliInit(fname, false);
+
+      int ecode = qt.executeClient(fname);
+      if (ecode != 0) {
+        qt.failed(ecode, fname, debugHint);
+      }
+      ecode = qt.checkCliDriverResults(fname);
+      if (ecode != 0) {
+        qt.failedDiff(ecode, fname, debugHint);
+      }
+    } catch (Throwable e) {
+      qt.failed(e, fname, debugHint);
+    }
+
+    long elapsedTime = System.currentTimeMillis() - startTime;
+    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime / 1000 + "s");
+    assertTrue("Test passed", true);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/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 b43c4a7..358ba51 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
@@ -77,6 +77,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hive.cli.CliDriver;
 import org.apache.hadoop.hive.cli.CliSessionState;
+import org.apache.hadoop.hive.cli.control.AbstractCliConfig;
 import org.apache.hadoop.hive.common.io.CachingPrintStream;
 import org.apache.hadoop.hive.common.io.DigestPrintStream;
 import org.apache.hadoop.hive.common.io.SortAndDigestPrintStream;
@@ -159,7 +160,7 @@ public class QTestUtil {
   private final Set<String> qNoSessionReuseQuerySet;
   private final Set<String> qJavaVersionSpecificOutput;
   private static final String SORT_SUFFIX = ".sorted";
-  public static final HashSet<String> srcTables = new HashSet<String>();
+  private final HashSet<String> srcTables;
   private static MiniClusterType clusterType = MiniClusterType.none;
   private ParseDriver pd;
   protected Hive db;
@@ -189,16 +190,21 @@ public class QTestUtil {
   }
   private HBaseTestingUtility utility;
 
-  static {
-    for (String srcTable : System.getProperty("test.src.tables", "").trim().split(",")) {
+  HashSet<String> getSrcTables() {
+    HashSet<String> srcTables = new HashSet<String>();
+    // FIXME: moved default value to here...for now
+    // i think this features is never really used from the command line
+    String defaultTestSrcTables = "src,src1,srcbucket,srcbucket2,src_json,src_thrift,src_sequencefile,srcpart,alltypesorc,src_hbase,cbo_t1,cbo_t2,cbo_t3,src_cbo,part,lineitem";
+    for (String srcTable : System.getProperty("test.src.tables", defaultTestSrcTables).trim().split(",")) {
       srcTable = srcTable.trim();
       if (!srcTable.isEmpty()) {
         srcTables.add(srcTable);
       }
     }
     if (srcTables.isEmpty()) {
-      throw new AssertionError("Source tables cannot be empty");
+      throw new RuntimeException("Source tables cannot be empty");
     }
+    return srcTables;
   }
 
   public HiveConf getConf() {
@@ -355,7 +361,7 @@ public class QTestUtil {
 
   private String getKeyProviderURI() {
     // Use the target directory if it is not specified
-    String HIVE_ROOT = QTestUtil.ensurePathEndsInSlash(System.getProperty("hive.root"));
+    String HIVE_ROOT = AbstractCliConfig.HIVE_ROOT;
     String keyDir = HIVE_ROOT + "ql/target/";
 
     // put the jks file in the current test path only for test purpose
@@ -385,7 +391,9 @@ public class QTestUtil {
     this.outDir = outDir;
     this.logDir = logDir;
     this.useHBaseMetastore = useHBaseMetastore;
+    this.srcTables=getSrcTables();
 
+    // HIVE-14443 move this fall-back logic to CliConfigs
     if (confDir != null && !confDir.isEmpty()) {
       HiveConf.setHiveSiteLocation(new URL("file://"+ new File(confDir).toURI().getPath() + "/hive-site.xml"));
       System.out.println("Setting hive-site: "+HiveConf.getHiveSiteLocation());
@@ -1008,7 +1016,7 @@ public class QTestUtil {
     cliDriver = new CliDriver();
 
     if (tname.equals("init_file.q")) {
-      ss.initFiles.add("../../data/scripts/test_init_file.sql");
+      ss.initFiles.add(AbstractCliConfig.HIVE_ROOT + "/data/scripts/test_init_file.sql");
     }
     cliDriver.processInitFiles(ss);
 
@@ -2040,10 +2048,10 @@ public class QTestUtil {
         LOG.debug("Connected to metastore database ");
       }
 
-      String mdbPath =   "../../data/files/tpcds-perf/metastore_export/";
+      String mdbPath =   AbstractCliConfig.HIVE_ROOT + "/data/files/tpcds-perf/metastore_export/";
 
       // Setup the table column stats
-      BufferedReader br = new BufferedReader(new FileReader(new File("../../metastore/scripts/upgrade/derby/022-HIVE-11107.derby.sql")));
+      BufferedReader br = new BufferedReader(new FileReader(new File(AbstractCliConfig.HIVE_ROOT + "/metastore/scripts/upgrade/derby/022-HIVE-11107.derby.sql")));
       String command;
 
       s.execute("DROP TABLE APP.TABLE_PARAMS");

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/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
new file mode 100644
index 0000000..a74f293
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/parse/CoreParseNegative.java
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.parse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.hadoop.hive.cli.control.AbstractCliConfig;
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+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;
+
+public class CoreParseNegative extends CliAdapter{
+
+  private static QTestUtil qt;
+
+  static CliConfigs.ParseNegativeConfig cliConfig = new CliConfigs.ParseNegativeConfig();
+
+  public CoreParseNegative(AbstractCliConfig testCliConfig) {
+    super(testCliConfig);
+  }
+
+  @Override
+  @BeforeClass
+  public void beforeClass() {
+    MiniClusterType miniMR = cliConfig.getClusterType();
+    String initScript = cliConfig.getInitScript();
+    String cleanupScript = cliConfig.getCleanupScript();
+
+    try {
+      String hadoopVer = cliConfig.getHadoopVersion();
+      qt = new QTestUtil((cliConfig.getResultsDir()), (cliConfig.getLogDir()), miniMR, null,
+          hadoopVer,
+       initScript, cleanupScript, false, false);
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      throw new RuntimeException("Unexpected exception in static initialization",e);
+    }
+  }
+
+  @Override
+  public void setUp() {
+  }
+
+  @Override
+  @After
+  public void tearDown() {
+    try {
+      qt.clearPostTestEffects();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in tearDown");
+    }
+  }
+
+  @Override
+  @AfterClass
+  public void shutdown() throws Exception {
+    try {
+      qt.shutdown();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      throw new RuntimeException("Unexpected exception in shutdown",e);
+    }
+  }
+
+  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();
+    try {
+      System.err.println("Begin query: " + fname);
+
+      qt.addFile(fpath);
+
+      qt.init(fname);
+      ASTNode tree = qt.parseQuery(fname);
+      List<Task<? extends Serializable>> tasks = qt.analyzeAST(tree);
+      fail("Unexpected success for query: " + fname + debugHint);
+    }
+    catch (ParseException pe) {
+      int ecode = qt.checkNegativeResults(fname, pe);
+      if (ecode != 0) {
+        qt.failed(ecode, fname, debugHint);
+      }
+    }
+    catch (SemanticException se) {
+      int ecode = qt.checkNegativeResults(fname, se);
+      if (ecode != 0) {
+        qt.failedDiff(ecode, fname, debugHint);
+      }
+    }
+    catch (Throwable e) {
+      qt.failed(e, fname, debugHint);
+    }
+
+    long elapsedTime = System.currentTimeMillis() - startTime;
+    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
+    assertTrue("Test passed", true);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 93a877e..b05a2dc 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1044,6 +1044,11 @@
             <java.security.krb5.conf>${test.tmp.dir}/conf/krb5.conf</java.security.krb5.conf>
             <!-- Required by spark to work around SPARK-14958 -->
             <antlr.version>${antlr.version}</antlr.version>
+            <qfile>${qfile}</qfile>
+            <initScript>${initScript}</initScript>
+            <clustermode>${clustermode}</clustermode>
+            <qfile_regex>${qfile_regex}</qfile_regex>
+            <run_disabled>${run_disabled}</run_disabled>
           </systemPropertyVariables>
         </configuration>
       </plugin>

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/ql/src/test/templates/TestBeeLineDriver.vm
----------------------------------------------------------------------
diff --git a/ql/src/test/templates/TestBeeLineDriver.vm b/ql/src/test/templates/TestBeeLineDriver.vm
deleted file mode 100644
index 563d7fd..0000000
--- a/ql/src/test/templates/TestBeeLineDriver.vm
+++ /dev/null
@@ -1,160 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.beeline.util;
-
-import static org.junit.Assert.fail;
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.*;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.QTestUtil;
-import org.apache.hive.service.server.HiveServer2;
-import org.apache.hive.testutils.junit.runners.ConcurrentTestRunner;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-@RunWith(ConcurrentTestRunner.class)
-public class $className {
-  private static final String hiveRootDirectory = "$hiveRootDir";
-  private static final String queryDirectory = "$queryDir";
-  private static final String logDirectory = "$logDir";
-  private static final String resultsDirectory = "$resultsDir";
-  private static boolean overwrite = false;
-  private static String scratchDirectory;
-  private static QTestUtil.QTestSetup miniZKCluster = null;
-
-  private static HiveServer2 hiveServer2;
-
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-    HiveConf hiveConf = new HiveConf();
-    hiveConf.logVars(System.err);
-    System.err.flush();
-
-    scratchDirectory = hiveConf.getVar(SCRATCHDIR);
-
-    String testOutputOverwrite = System.getProperty("test.output.overwrite");
-    if (testOutputOverwrite != null && "true".equalsIgnoreCase(testOutputOverwrite)) {
-      overwrite = true;
-    }
-
-    miniZKCluster = new QTestUtil.QTestSetup();
-    miniZKCluster.preTest(hiveConf);
-
-    System.setProperty("hive.zookeeper.quorum",
-        hiveConf.get("hive.zookeeper.quorum"));
-    System.setProperty("hive.zookeeper.client.port",
-        hiveConf.get("hive.zookeeper.client.port"));
-    
-    String disableserver = System.getProperty("test.service.disable.server");
-    if (null != disableserver && disableserver.equalsIgnoreCase("true")) {
-      System.err.println("test.service.disable.server=true "
-        + "Skipping HiveServer2 initialization!");
-      return;
-    }
-
-    hiveServer2 = new HiveServer2();
-    hiveServer2.init(hiveConf);
-    System.err.println("Starting HiveServer2...");
-    hiveServer2.start();
-    Thread.sleep(5000);
-  }
-
-
-  @AfterClass
-  public static void afterClass() {
-    try {
-      if (hiveServer2 != null) {
-        System.err.println("Stopping HiveServer2...");
-        hiveServer2.stop();
-      }
-    } catch (Throwable t) {
-      t.printStackTrace();
-    }
-    
-    if (miniZKCluster != null) {
-      try {
-        miniZKCluster.tearDown();
-      } catch (Exception e) {
-        e.printStackTrace();
-      }
-    }
-  }
-
-
-  /*
-  public $className() {
-  }
-  */
-
-  protected static void runTest(String qFileName) throws Exception {
-    QFileClient qClient = new QFileClient(new HiveConf(), hiveRootDirectory,
-        queryDirectory, logDirectory, resultsDirectory)
-    .setQFileName(qFileName)
-    .setUsername("user")
-    .setPassword("password")
-    .setJdbcUrl("jdbc:hive2://localhost:10000")
-    .setJdbcDriver("org.apache.hive.jdbc.HiveDriver")
-    .setTestDataDirectory(hiveRootDirectory + "/data/files")
-    .setTestScriptDirectory(hiveRootDirectory + "/data/scripts");
-
-    long startTime = System.currentTimeMillis();
-    System.err.println(">>> STARTED " + qFileName
-        + " (Thread " + Thread.currentThread().getName() + ")");
-    try {
-      qClient.run();
-    } catch (Exception e) {
-      System.err.println(">>> FAILED " + qFileName + " with exception:");
-      e.printStackTrace();
-      throw e;
-    }
-    long elapsedTime = (System.currentTimeMillis() - startTime)/1000;
-    String time = "(" + elapsedTime + "s)";
-    
-    if (qClient.compareResults()) {
-      System.err.println(">>> PASSED " + qFileName + " " + time);
-    } else {
-      if (qClient.hasErrors()) {
-        System.err.println(">>> FAILED " + qFileName + " (ERROR) " + time);
-        fail();
-      }
-      if (overwrite) {
-        System.err.println(">>> PASSED " + qFileName + " (OVERWRITE) " + time);
-        qClient.overwriteResults();
-      } else {
-        System.err.println(">>> FAILED " + qFileName + " (DIFF) " + time);
-        fail();
-      }
-    }
-  }
-
-  
-#foreach ($qf in $qfiles)
-  #set ($fname = $qf.getName())
-  #set ($eidx = $fname.indexOf('.'))
-  #set ($tname = $fname.substring(0, $eidx))
-  @Test
-  public void testBeeLineDriver_$tname() throws Exception {
-    runTest("$fname");
-  }
-#end
-
-}
-
-

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/ql/src/test/templates/TestCliDriver.vm
----------------------------------------------------------------------
diff --git a/ql/src/test/templates/TestCliDriver.vm b/ql/src/test/templates/TestCliDriver.vm
deleted file mode 100644
index 0ccedce..0000000
--- a/ql/src/test/templates/TestCliDriver.vm
+++ /dev/null
@@ -1,143 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.cli;
-
-import org.apache.hadoop.hive.ql.QTestUtil;
-import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class $className {
-
-  private static final String HIVE_ROOT = QTestUtil.ensurePathEndsInSlash(System.getProperty("hive.root"));
-  private static QTestUtil qt;
-
-  static {
-
-    MiniClusterType miniMR = MiniClusterType.valueForString("$clusterMode");
-    String hiveConfDir = "$hiveConfDir";
-    String initScript = "$initScript";
-    String cleanupScript = "$cleanupScript";
-    boolean useHBaseMetastore = Boolean.valueOf("$useHBaseMetastore");
-    try {
-      String hadoopVer = "$hadoopVersion";
-      if (!hiveConfDir.isEmpty()) {
-        hiveConfDir = HIVE_ROOT + hiveConfDir;
-      }
-      qt = new QTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR,
-      hiveConfDir, hadoopVer, initScript, cleanupScript, useHBaseMetastore, true);
-
-      // do a one time initialization
-      qt.cleanUp();
-      qt.createSources();
-
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in static initialization: "+e.getMessage());
-    }
-  }
-
-  @Before
-  public void setUp() {
-    try {
-      qt.clearTestSideEffects();
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in setup");
-    }
-  }
-
-  @After
-  public void tearDown() {
-    try {
-      qt.clearPostTestEffects();
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in tearDown");
-    }
-  }
-
-  @AfterClass
-  public static void shutdown() throws Exception {
-    try {
-      qt.shutdown();
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in shutdown");
-    }
-  }
-
-  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.";
-
-#foreach ($qf in $qfiles)
-  #set ($fname = $qf.getName())
-  #set ($eidx = $fname.indexOf('.'))
-  #set ($tname = $fname.substring(0, $eidx))
-  #set ($fpath = $qfilesMap.get($fname))
-  @Test
-  public void testCliDriver_$tname() throws Exception {
-    runTest("$tname", "$fname", (HIVE_ROOT + "$fpath"));
-  }
-
-#end
-
-  private void runTest(String tname, String fname, String fpath) throws Exception {
-    long startTime = System.currentTimeMillis();
-    try {
-      System.err.println("Begin query: " + fname);
-
-      qt.addFile(fpath);
-
-      if (qt.shouldBeSkipped(fname)) {
-        System.err.println("Test " + fname + " skipped");
-        return;
-      }
-
-      qt.cliInit(fname, false);
-      int ecode = qt.executeClient(fname);
-      if (ecode != 0) {
-        qt.failed(ecode, fname, debugHint);
-      }
-      ecode = qt.checkCliDriverResults(fname);
-      if (ecode != 0) {
-        qt.failedDiff(ecode, fname, debugHint);
-      }
-    }
-    catch (Throwable e) {
-      qt.failed(e, fname, debugHint);
-    }
-
-    long elapsedTime = System.currentTimeMillis() - startTime;
-    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
-    assertTrue("Test passed", true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/ql/src/test/templates/TestCompareCliDriver.vm
----------------------------------------------------------------------
diff --git a/ql/src/test/templates/TestCompareCliDriver.vm b/ql/src/test/templates/TestCompareCliDriver.vm
deleted file mode 100644
index 8d4e964..0000000
--- a/ql/src/test/templates/TestCompareCliDriver.vm
+++ /dev/null
@@ -1,162 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.cli;
-
-import java.io.*;
-import java.util.*;
-
-import org.apache.hadoop.hive.ql.QTestUtil;
-import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.fail;
-
-public class $className {
-
-  private static final String HIVE_ROOT = QTestUtil.ensurePathEndsInSlash(System.getProperty("hive.root"));
-  private static QTestUtil qt;
-
-  static {
-
-    MiniClusterType miniMR = MiniClusterType.valueForString("$clusterMode");
-    String hiveConfDir = "$hiveConfDir";
-    String initScript = "$initScript";
-    String cleanupScript = "$cleanupScript";
-    try {
-      String hadoopVer = "$hadoopVersion";
-      if (!hiveConfDir.isEmpty()) {
-        hiveConfDir = HIVE_ROOT + hiveConfDir;
-      }
-      qt = new QTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR,
-      hiveConfDir, hadoopVer, initScript, cleanupScript, false, false);
-
-      // do a one time initialization
-      qt.cleanUp();
-      qt.createSources();
-
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in static initialization");
-    }
-  }
-
-  @Before
-  public void setUp() {
-    try {
-      qt.clearTestSideEffects();
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in setup");
-    }
-  }
-
-  @After
-  public void tearDown() {
-    try {
-      qt.clearPostTestEffects();
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in tearDown");
-    }
-  }
-
-  @AfterClass
-  public static void shutdown() throws Exception {
-    try {
-      qt.shutdown();
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in shutdown");
-    }
-  }
-
-  private Map<String, List<String>> versionFiles = new HashMap<String, List<String>>();
-
-  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.";
-
-#foreach ($qf in $qfiles)
-  #set ($fname = $qf.getName())
-  #set ($eidx = $fname.indexOf('.'))
-  #set ($tname = $fname.substring(0, $eidx))
-  #set ($fpath = $qfilesMap.get($fname))
-  @Test
-  public void testCompareCliDriver_$tname() throws Exception {
-    runTest("$tname", "$fname", (HIVE_ROOT + "$fpath"));
-  }
-
-#end
-
-  private void runTest(String tname, String fname, String fpath) throws Exception {
-    final String queryDirectory = HIVE_ROOT + "$queryDir";
-
-    long startTime = System.currentTimeMillis();
-    try {
-      System.err.println("Begin query: " + fname);
-      // TODO: versions could also be picked at build time.
-      List<String> versionFiles = QTestUtil.getVersionFiles(queryDirectory, tname);
-      if (versionFiles.size() < 2) {
-        fail("Cannot run " + tname + " with only " + versionFiles.size() + " versions");
-      }
-
-      qt.addFile(fpath);
-      for (String versionFile : versionFiles) {
-        qt.addFile(new File(queryDirectory, versionFile), true);
-      }
-
-      if (qt.shouldBeSkipped(fname)) {
-        return;
-      }
-
-      int ecode = 0;
-      List<String> outputs = new ArrayList<String>(versionFiles.size());
-      for (String versionFile : versionFiles) {
-        // 1 for "_" after tname; 3 for ".qv" at the end. Version is in between.
-        String versionStr = versionFile.substring(tname.length() + 1, versionFile.length() - 3);
-        outputs.add(qt.cliInit(tname + "." + versionStr, false));
-        // TODO: will this work?
-        ecode = qt.executeClient(versionFile, fname);
-        if (ecode != 0) {
-          qt.failed(ecode, fname, debugHint);
-        }
-      }
-
-      ecode = qt.checkCompareCliDriverResults(fname, outputs);
-      if (ecode != 0) {
-        qt.failedDiff(ecode, fname, debugHint);
-      }
-    }
-    catch (Throwable e) {
-      qt.failed(e, fname, debugHint);
-    }
-
-    long elapsedTime = System.currentTimeMillis() - startTime;
-    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/ql/src/test/templates/TestNegativeCliDriver.vm
----------------------------------------------------------------------
diff --git a/ql/src/test/templates/TestNegativeCliDriver.vm b/ql/src/test/templates/TestNegativeCliDriver.vm
deleted file mode 100644
index 592d64f..0000000
--- a/ql/src/test/templates/TestNegativeCliDriver.vm
+++ /dev/null
@@ -1,147 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.cli;
-
-import org.apache.hadoop.hive.ql.QTestUtil;
-import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class $className {
-
-  private static final String HIVE_ROOT = QTestUtil.ensurePathEndsInSlash(System.getProperty("hive.root"));
-  private static QTestUtil qt;
-
-  static {
-    MiniClusterType miniMR = MiniClusterType.valueForString("$clusterMode");
-    String hiveConfDir = "$hiveConfDir";
-    String initScript = "$initScript";
-    String cleanupScript = "$cleanupScript";
-
-    try {
-      String hadoopVer = "$hadoopVersion";
-      if (!hiveConfDir.isEmpty()) {
-        hiveConfDir = HIVE_ROOT + hiveConfDir;
-      }
-      qt = new QTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR,
-       hiveConfDir, hadoopVer, initScript, cleanupScript, false, false);
-      // do a one time initialization
-      qt.cleanUp();
-      qt.createSources();
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in static initialization");
-    }
-  }
-
-  @Before
-  public void setUp() {
-    try {
-      qt.clearTestSideEffects();
-    } catch (Throwable e) {
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in setup");
-    }
-  }
-
-  @After
-  public void tearDown() {
-    try {
-      qt.clearPostTestEffects();
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in tearDown");
-    }
-  }
-
-  @AfterClass
-  public static void shutdown() throws Exception {
-    try {
-      qt.shutdown();
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in shutdown");
-    }
-  }
-
-  /**
-   * 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, "
-     + "or check ./ql/target/surefire-reports or ./itests/qtest/target/surefire-reports/ for specific test cases logs.";
-
-#foreach ($qf in $qfiles)
-  #set ($fname = $qf.getName())
-  #set ($eidx = $fname.indexOf('.'))
-  #set ($tname = $fname.substring(0, $eidx))
-  #set ($fpath = $qfilesMap.get($fname))
-  @Test
-  public void testNegativeCliDriver_$tname() throws Exception {
-    runTest("$tname", "$fname", (HIVE_ROOT + "$fpath"));
-  }
-
-#end
-
-  private void runTest(String tname, String fname, String fpath) throws Exception {
-    long startTime = System.currentTimeMillis();
-    try {
-      System.err.println("Begin query: " + fname);
-
-      qt.addFile(fpath);
-
-      if (qt.shouldBeSkipped(fname)) {
-        System.err.println("Test " + fname + " skipped");
-        return;
-      }
-
-      qt.cliInit(fname, false);
-      int ecode = qt.executeClient(fname);
-      if (ecode == 0) {
-        qt.failed(fname, debugHint);
-      }
-
-      ecode = qt.checkCliDriverResults(fname);
-      if (ecode != 0) {
-        qt.failedDiff(ecode, fname, debugHint);
-      }
-    }
-    catch (Throwable e) {
-      qt.failed(e, fname, debugHint);
-    }
-
-    long elapsedTime = System.currentTimeMillis() - startTime;
-    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
-    assertTrue("Test passed", true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/ql/src/test/templates/TestParseNegative.vm
----------------------------------------------------------------------
diff --git a/ql/src/test/templates/TestParseNegative.vm b/ql/src/test/templates/TestParseNegative.vm
deleted file mode 100755
index 9500ece..0000000
--- a/ql/src/test/templates/TestParseNegative.vm
+++ /dev/null
@@ -1,127 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.ql.parse;
-
-import java.io.*;
-import java.util.*;
-
-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.Test;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class $className {
-
-  private static final String HIVE_ROOT = QTestUtil.ensurePathEndsInSlash(System.getProperty("hive.root"));
-  private static QTestUtil qt;
- 
-  static {
-
-    MiniClusterType miniMR = MiniClusterType.valueForString("$clusterMode");
-    String initScript = "$initScript";
-    String cleanupScript = "$cleanupScript";
-
-    try {
-      String hadoopVer = "$hadoopVersion";
-      qt = new QTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR, null, hadoopVer,
-       initScript, cleanupScript, false, false);
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in static initialization");
-    }
-  }
-
-  @After
-  public void tearDown() {
-    try {
-      qt.clearPostTestEffects();
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in tearDown");
-    }
-  }
-
-  @AfterClass
-  public static void shutdown() throws Exception {
-    try {
-      qt.shutdown();
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in shutdown");
-    }
-  }
-
-  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.";
-
-#foreach ($qf in $qfiles)
-  #set ($fname = $qf.getName())
-  #set ($eidx = $fname.indexOf('.'))
-  #set ($tname = $fname.substring(0, $eidx))
-  #set ($fpath = $qfilesMap.get($fname))
-  @Test
-  public void testParseNegative_$tname() throws Exception {
-    runTest("$tname", "$fname", (HIVE_ROOT + "$fpath"));
-  }
-
-#end
-
-  private void runTest(String tname, String fname, String fpath) throws Exception {
-    long startTime = System.currentTimeMillis();
-    try {
-      System.err.println("Begin query: " + fname);
-
-      qt.addFile(fpath);
-
-      qt.init(fname);
-      ASTNode tree = qt.parseQuery(fname);
-      List<Task<? extends Serializable>> tasks = qt.analyzeAST(tree);
-      fail("Unexpected success for query: " + fname + debugHint);
-    }
-    catch (ParseException pe) {
-      int ecode = qt.checkNegativeResults(fname, pe);
-      if (ecode != 0) {
-        qt.failed(ecode, fname, debugHint);
-      }
-    }
-    catch (SemanticException se) {
-      int ecode = qt.checkNegativeResults(fname, se);
-      if (ecode != 0) {
-        qt.failedDiff(ecode, fname, debugHint);
-      }
-    }
-    catch (Throwable e) {
-      qt.failed(e, fname, debugHint);
-    }
-
-    long elapsedTime = System.currentTimeMillis() - startTime;
-    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
-    assertTrue("Test passed", true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/ql/src/test/templates/TestPerfCliDriver.vm
----------------------------------------------------------------------
diff --git a/ql/src/test/templates/TestPerfCliDriver.vm b/ql/src/test/templates/TestPerfCliDriver.vm
deleted file mode 100644
index d2946cb..0000000
--- a/ql/src/test/templates/TestPerfCliDriver.vm
+++ /dev/null
@@ -1,177 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.cli;
-
-import junit.framework.Test;
-import junit.framework.TestCase;
-import junit.framework.TestSuite;
-
-import java.io.*;
-import java.util.*;
-import java.nio.*;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-
-import org.apache.hadoop.hive.ql.QTestUtil;
-import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
-import org.apache.hadoop.hive.ql.session.SessionState;
-
-/**
- This is the TestPerformance Cli Driver for integrating performance regression tests
- as part of the Hive Unit tests. 
- Currently this includes support for :
- 1. Running explain plans for TPCDS workload (non-partitioned dataset)  on 30TB scaleset.
- TODO :
- 1. Support for partitioned data set
- 2. Use HBase Metastore instead of Derby 
-
-This suite differs from TestCliDriver w.r.t the fact that we modify the underlying metastore
-database to reflect the dataset before running the queries.
-*/
-public class $className extends TestCase {
-
-  private static final String HIVE_ROOT = QTestUtil.ensurePathEndsInSlash(System.getProperty("hive.root"));
-  private static QTestUtil qt;
-
-  public static class TestPerfCliDriverAddTestFromQFiles implements QTestUtil.SuiteAddTestFunctor {
-    public void addTestToSuite(TestSuite suite, Object setup, String tName) {
-      suite.addTest(new $className("testPerfCliDriver_"+tName));
-    }
-  }
-
-  static {
-    System.setProperty("datanucleus.schema.autoCreateAll", "true");
-    System.setProperty("hive.metastore.schema.verification", "false");
-    MiniClusterType miniMR = MiniClusterType.valueForString("$clusterMode");
-    String hiveConfDir = "$hiveConfDir";
-    String initScript = "$initScript";
-    String cleanupScript = "$cleanupScript";
-    try {
-      String hadoopVer = "$hadoopVersion";
-      if (!hiveConfDir.isEmpty()) {
-        hiveConfDir = HIVE_ROOT + hiveConfDir;
-      }
-      qt = new QTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR,
-      hiveConfDir, hadoopVer, initScript, cleanupScript, false, false);
-
-      // do a one time initialization
-      qt.cleanUp();
-      qt.createSources();
-      // Manually modify the underlying metastore db to reflect statistics corresponding to
-      // the 30TB TPCDS scale set. This way the optimizer will generate plans for a 30 TB set.
-      QTestUtil.setupMetaStoreTableColumnStatsFor30TBTPCDSWorkload(qt.getConf());
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in static initialization: "+e.getMessage());
-    }
-  }
-
-  public $className(String name) {
-    super(name);
-  }
-
-  /**
-   * Dummy last test. This is only meant to shutdown qt
-   */
-  public void testPerfCliDriver_shutdown() {
-    System.err.println ("Cleaning up " + "$className");
-  }
-
-  @Override
-  protected void tearDown() {
-    try {
-      qt.clearPostTestEffects();
-      if (getName().equals("testPerfCliDriver_shutdown"))
-        qt.shutdown();
-    }
-    catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in tearDown");
-    }
-  }
-
-  public static Test suite() {
-    Set<String> qFilesToExecute = new HashSet<String>();
-    String qFiles = System.getProperty("qfile", "").trim();
-    if(!qFiles.isEmpty()) {
-      for(String qFile : qFiles.split(",")) {
-        qFile = qFile.trim();
-        if(!qFile.isEmpty()) {
-          qFilesToExecute.add(qFile);
-        }
-      }
-    }
-
-    TestSuite suite = new TestSuite();
-
-    QTestUtil.addTestsToSuiteFromQfileNames("$qFileNamesFile", qFilesToExecute,
-      suite, null, new TestPerfCliDriverAddTestFromQFiles());
-    suite.addTest(new $className("testPerfCliDriver_shutdown"));
-    return suite;
-  }
-
-  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.";
-
-#foreach ($qf in $qfiles)
-  #set ($fname = $qf.getName())
-  #set ($eidx = $fname.indexOf('.'))
-  #set ($tname = $fname.substring(0, $eidx))
-  #set ($fpath = $qfilesMap.get($fname))
-  public void testPerfCliDriver_$tname() throws Exception {
-    runTest("$tname", "$fname", (HIVE_ROOT + "$fpath"));
-  }
-
-#end
-
-  private void runTest(String tname, String fname, String fpath) throws Exception {
-    long startTime = System.currentTimeMillis();
-    try {
-      System.err.println("Begin query: " + fname);
-
-      qt.addFile(fpath);
-
-      if (qt.shouldBeSkipped(fname)) {
-        return;
-      }
-
-      qt.cliInit(fname, false);
-
-      int ecode = qt.executeClient(fname);
-      if (ecode != 0) {
-        qt.failed(ecode, fname, debugHint);
-      }
-      ecode = qt.checkCliDriverResults(fname);
-      if (ecode != 0) {
-        qt.failedDiff(ecode, fname, debugHint);
-      }
-    }
-    catch (Throwable e) {
-      qt.failed(e, fname, debugHint);
-    }
-
-    long elapsedTime = System.currentTimeMillis() - startTime;
-    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
-    assertTrue("Test passed", true);
-  }
-}


[3/3] hive git commit: HIVE-14444 Upgrade qtest execution framework to junit4 - migrate most of them (Zoltan Haindrich via Ashutosh Chauhan)

Posted by ha...@apache.org.
HIVE-14444 Upgrade qtest execution framework to junit4 - migrate most of them (Zoltan Haindrich via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master
Commit: 1f7e10c031cb63b80d9de950004108d70cd57afc
Parents: bfdc74f
Author: Zoltan Haindrich <ki...@rxd.hu>
Authored: Fri Aug 12 17:12:03 2016 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Fri Aug 12 17:12:03 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ant/QTestGenTask.java    | 585 -------------------
 ant/src/org/apache/hadoop/hive/ant/antlib.xml   |   2 -
 .../src/test/templates/TestHBaseCliDriver.vm    | 144 -----
 .../templates/TestHBaseNegativeCliDriver.vm     | 119 ----
 itests/qtest-accumulo/pom.xml                   |   5 +-
 itests/qtest-spark/pom.xml                      |  54 --
 .../hive/cli/TestMiniSparkOnYarnCliDriver.java  |  45 ++
 .../hadoop/hive/cli/TestSparkCliDriver.java     |  45 ++
 .../hive/cli/TestSparkNegativeCliDriver.java    |  45 ++
 itests/qtest/pom.xml                            | 267 +--------
 .../hive/cli/ContribNegativeCliDriver.java      |  62 ++
 .../hive/cli/DisabledTestBeeLineDriver.java     |  62 ++
 .../apache/hadoop/hive/cli/DummyCliDriver.java  |  62 ++
 .../apache/hadoop/hive/cli/TestCliDriver.java   |  62 ++
 .../hadoop/hive/cli/TestCompareCliDriver.java   |  62 ++
 .../hadoop/hive/cli/TestContribCliDriver.java   |  62 ++
 .../hive/cli/TestContribNegativeCliDriver.java  |  62 ++
 .../hive/cli/TestEncryptedHDFSCliDriver.java    |  62 ++
 .../hadoop/hive/cli/TestHBaseCliDriver.java     |  62 ++
 .../hive/cli/TestHBaseMinimrCliDriver.java      |  62 ++
 .../hive/cli/TestHBaseNegativeCliDriver.java    |  62 ++
 .../hadoop/hive/cli/TestMiniLlapCliDriver.java  |  62 ++
 .../hadoop/hive/cli/TestMiniTezCliDriver.java   |  62 ++
 .../hadoop/hive/cli/TestMinimrCliDriver.java    |  62 ++
 .../hadoop/hive/cli/TestNegativeCliDriver.java  |  62 ++
 .../hive/cli/TestNegativeMinimrCliDriver.java   |  62 ++
 .../hadoop/hive/cli/TestPerfCliDriver.java      |  62 ++
 .../hive/ql/parse/TestParseNegativeDriver.java  |  62 ++
 .../hive/cli/control/AbstractCliConfig.java     | 417 +++++++++++++
 .../hadoop/hive/cli/control/CliAdapter.java     | 107 ++++
 .../hadoop/hive/cli/control/CliConfigs.java     | 503 ++++++++++++++++
 .../hive/cli/control/CoreBeeLineDriver.java     | 171 ++++++
 .../hadoop/hive/cli/control/CoreCliDriver.java  | 137 +++++
 .../hive/cli/control/CoreCompareCliDriver.java  | 158 +++++
 .../hadoop/hive/cli/control/CoreDummy.java      |  57 ++
 .../hive/cli/control/CoreHBaseCliDriver.java    | 137 +++++
 .../cli/control/CoreHBaseNegativeCliDriver.java | 121 ++++
 .../hive/cli/control/CoreNegativeCliDriver.java | 139 +++++
 .../hive/cli/control/CorePerfCliDriver.java     | 138 +++++
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |  24 +-
 .../hadoop/hive/ql/parse/CoreParseNegative.java | 132 +++++
 pom.xml                                         |   5 +
 ql/src/test/templates/TestBeeLineDriver.vm      | 160 -----
 ql/src/test/templates/TestCliDriver.vm          | 143 -----
 ql/src/test/templates/TestCompareCliDriver.vm   | 162 -----
 ql/src/test/templates/TestNegativeCliDriver.vm  | 147 -----
 ql/src/test/templates/TestParseNegative.vm      | 127 ----
 ql/src/test/templates/TestPerfCliDriver.vm      | 177 ------
 48 files changed, 3492 insertions(+), 2097 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java
----------------------------------------------------------------------
diff --git a/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java b/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java
deleted file mode 100644
index f372d7c..0000000
--- a/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java
+++ /dev/null
@@ -1,585 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ant;
-
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileFilter;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.regex.Pattern;
-import java.util.HashMap;
-
-import com.google.common.base.Splitter;
-import com.google.common.collect.Sets;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.tools.ant.BuildException;
-import org.apache.tools.ant.Task;
-import org.apache.velocity.app.VelocityEngine;
-import org.apache.velocity.Template;
-import org.apache.velocity.VelocityContext;
-import org.apache.velocity.exception.MethodInvocationException;
-import org.apache.velocity.exception.ParseErrorException;
-import org.apache.velocity.exception.ResourceNotFoundException;
-import org.apache.velocity.runtime.RuntimeConstants;
-
-public class QTestGenTask extends Task {
-   private static final Splitter TEST_SPLITTER = Splitter.onPattern("[, ]")
-       .trimResults()
-       .omitEmptyStrings();
-
-  public class IncludeFilter implements FileFilter {
-
-    Set<String> includeOnly;
-    public IncludeFilter(Set<String> includeOnly) {
-      this.includeOnly = includeOnly;
-    }
-
-    public boolean accept(File fpath) {
-      return includeOnly == null || includeOnly.contains(fpath.getName());
-    }
-  }
-
-  public class QFileFilter extends IncludeFilter {
-
-    public QFileFilter(Set<String> includeOnly) {
-      super(includeOnly);
-    }
-
-    public boolean accept(File fpath) {
-      if (!super.accept(fpath)) {
-        return false;
-      }
-      if (fpath.isDirectory() ||
-          !fpath.getName().endsWith(".q")) {
-        return false;
-      }
-      return true;
-    }
-  }
-
-  public class DisabledQFileFilter extends IncludeFilter {
-    public DisabledQFileFilter(Set<String> includeOnly) {
-      super(includeOnly);
-    }
-
-    public boolean accept(File fpath) {
-      if (!super.accept(fpath)) {
-        return false;
-      }
-      return !fpath.isDirectory() && fpath.getName().endsWith(".q.disabled");
-    }
-  }
-
-  public class QFileRegexFilter implements FileFilter {
-    Pattern filterPattern;
-    public QFileRegexFilter(String filter) {
-      filterPattern = Pattern.compile(filter);
-    }
-
-    public boolean accept(File filePath) {
-      if (filePath.isDirectory() ||
-          !filePath.getName().endsWith(".q")) {
-        return false;
-      }
-      String testName = StringUtils.chomp(filePath.getName(), ".q");
-      return filterPattern.matcher(testName).matches();
-    }
-  }
-
-  private List<String> templatePaths = new ArrayList<String>();
-
-  private String hiveRootDirectory;
-
-  private String outputDirectory;
-
-  private String queryDirectory;
-
-  private String queryFile;
-
-  private String includeQueryFile;
-
-  private String excludeQueryFile;
-
-  private String queryFileRegex;
-
-  private String resultsDirectory;
-
-  private String logDirectory;
-
-  private String tezDirectory;
-
-  private String template;
-
-  private String className;
-
-  private String logFile;
-
-  private String clusterMode;
-
-  private String hiveConfDir;
-
-  private String runDisabled;
-
-  private String hadoopVersion;
-
-  private String initScript;
-
-  private String cleanupScript;
-
-  private String useHBaseMetastore;
-
-  public void setHadoopVersion(String ver) {
-    this.hadoopVersion = ver;
-  }
-
-  public String getHadoopVersion() {
-    return hadoopVersion;
-  }
-
-  public void setHiveConfDir(String hiveConfDir) {
-    this.hiveConfDir = hiveConfDir;
-  }
-
-  public String getHiveConfDir() {
-    return hiveConfDir;
-  }
-
-  public void setClusterMode(String clusterMode) {
-    this.clusterMode = clusterMode;
-  }
-
-  public String getClusterMode() {
-    return clusterMode;
-  }
-
-  public void setRunDisabled(String runDisabled) {
-    this.runDisabled = runDisabled;
-  }
-
-  public String getRunDisabled() {
-    return runDisabled;
-  }
-
-  public void setLogFile(String logFile) {
-    this.logFile = logFile;
-  }
-
-  public String getLogFile() {
-    return logFile;
-  }
-
-  public void setClassName(String className) {
-    this.className = className;
-  }
-
-  public String getClassName() {
-    return className;
-  }
-
-  public void setTemplate(String template) {
-    this.template = template;
-  }
-
-  public String getTemplate() {
-    return template;
-  }
-
-  public String getInitScript() {
-    return initScript;
-  }
-
-  public void setInitScript(String initScript) {
-    this.initScript = initScript;
-  }
-
-  public String getCleanupScript() {
-    return cleanupScript;
-  }
-
-  public void setCleanupScript(String cleanupScript) {
-    this.cleanupScript = cleanupScript;
-  }
-
-  public String getUseHBaseMetastore() {
-    return useHBaseMetastore;
-  }
-
-  public void setUseHBaseMetastore(String useHBaseMetastore) {
-    this.useHBaseMetastore = useHBaseMetastore;
-  }
-
-  public void setHiveRootDirectory(File hiveRootDirectory) {
-    try {
-      this.hiveRootDirectory = hiveRootDirectory.getCanonicalPath();
-    } catch (IOException ioe) {
-      throw new BuildException(ioe);
-    }
-  }
-
-  public String getHiveRootDirectory() {
-    return hiveRootDirectory;
-  }
-
-  public void setTemplatePath(String templatePath) throws Exception {
-    templatePaths.clear();
-    for (String relativePath : TEST_SPLITTER.split(templatePath)) {
-      templatePaths.add(project.resolveFile(relativePath).getCanonicalPath());
-    }
-    System.out.println("Template Path:" + getTemplatePath());
-  }
-
-  public String getTemplatePath() {
-    return StringUtils.join(templatePaths, ",");
-  }
-
-  public void setOutputDirectory(File outputDirectory) {
-    try {
-      this.outputDirectory = outputDirectory.getCanonicalPath();
-    } catch (IOException ioe) {
-      throw new BuildException(ioe);
-    }
-  }
-
-  public String getOutputDirectory() {
-    return outputDirectory;
-  }
-
-  public void setLogDirectory(String logDirectory) {
-    this.logDirectory = logDirectory;
-  }
-
-  public String getLogDirectory() {
-    return logDirectory;
-  }
-
-  public void setResultsDirectory(String resultsDirectory) {
-    this.resultsDirectory = resultsDirectory;
-  }
-
-  public String getResultsDirectory() {
-    return resultsDirectory;
-  }
-
-  public void setQueryDirectory(String queryDirectory) {
-    this.queryDirectory = queryDirectory;
-  }
-
-  public String getQueryDirectory() {
-    return queryDirectory;
-  }
-
-  public void setQueryFile(String queryFile) {
-    this.queryFile = queryFile;
-  }
-
-  public String getQueryFile() {
-    return queryFile;
-  }
-
-  public String getIncludeQueryFile() {
-    return includeQueryFile;
-  }
-
-  public void setIncludeQueryFile(String includeQueryFile) {
-    this.includeQueryFile = includeQueryFile;
-  }
-
-  public void setExcludeQueryFile(String excludeQueryFile) {
-    this.excludeQueryFile = excludeQueryFile;
-  }
-
-  public String getExcludeQueryFile() {
-    return excludeQueryFile;
-  }
-
-  public void setQueryFileRegex(String queryFileRegex) {
-    this.queryFileRegex = queryFileRegex;
-  }
-
-  public String getQueryFileRegex() {
-    return queryFileRegex;
-  }
-
-  private String createAlternativeFile(File file) throws Exception {
-    String fileParentDir = file.getParent();
-    String fileName = file.getName();
-    int dotIndex = fileName.lastIndexOf('.');
-    String fileNameWithoutExtension = dotIndex == -1 ? fileName : fileName.substring(0, dotIndex);
-    String fileNameExtension = dotIndex == -1 ? "" : fileName.substring(dotIndex);
-
-    // If prefix length is < 3, File.createTempFile() will throw an IllegalArgumentException.
-    // We need to avoid this case.
-    if (fileNameWithoutExtension.length() < 3) {
-      fileNameWithoutExtension = fileNameWithoutExtension + "_tmp";
-    }
-    File alternativeFile = File.createTempFile(fileNameWithoutExtension, fileNameExtension,
-      new File(fileParentDir));
-    return alternativeFile.getCanonicalPath();
-  }
-
-  public void execute() throws BuildException {
-    if (getTemplatePath().equals("")) {
-      throw new BuildException("No templatePath attribute specified");
-    }
-
-    if (template == null) {
-      throw new BuildException("No template attribute specified");
-    }
-
-    if (outputDirectory == null) {
-      throw new BuildException("No outputDirectory specified");
-    }
-
-    if (queryDirectory == null && queryFile == null ) {
-      throw new BuildException("No queryDirectory or queryFile specified");
-    }
-
-    if (logDirectory == null) {
-      throw new BuildException("No logDirectory specified");
-    }
-
-    if (className == null) {
-      throw new BuildException("No className specified");
-    }
-
-    Set<String> includeOnly = null;
-    if (includeQueryFile != null && !includeQueryFile.isEmpty()) {
-      includeOnly = Sets.<String>newHashSet(TEST_SPLITTER.split(includeQueryFile));
-    }
-
-    List<File> qFiles;
-    HashMap<String, String> qFilesMap = new HashMap<String, String>();
-    File hiveRootDir = null;
-    File queryDir = null;
-    File outDir = null;
-    File resultsDir = null;
-    File logDir = null;
-
-    try {
-
-      System.out.println("Starting Generation of: " + className);
-      System.out.println("Include Files: " + includeQueryFile);
-      System.out.println("Excluded Files: " + excludeQueryFile);
-      System.out.println("Query Files: " + queryFile);
-      System.out.println("Query Files Regex: " + queryFileRegex);
-
-      // queryDirectory should not be null
-      queryDir = new File(queryDirectory);
-
-      // dedup file list
-      Set<File> testFiles = new HashSet<File>();
-      if (queryFile != null && !queryFile.equals("")) {
-        // The user may have passed a list of files - comma separated
-        for (String qFile : TEST_SPLITTER.split(queryFile)) {
-          if (null != queryDir) {
-            testFiles.add(new File(queryDir, qFile));
-          } else {
-            testFiles.add(new File(qFile));
-          }
-        }
-      } else if (queryFileRegex != null && !queryFileRegex.equals("")) {
-        for (String regex : TEST_SPLITTER.split(queryFileRegex)) {
-          testFiles.addAll(Arrays.asList(queryDir.listFiles(
-              new QFileRegexFilter(regex))));
-        }
-      } else if (runDisabled != null && runDisabled.equals("true")) {
-        testFiles.addAll(Arrays.asList(queryDir.listFiles(new DisabledQFileFilter(includeOnly))));
-      } else {
-        testFiles.addAll(Arrays.asList(queryDir.listFiles(new QFileFilter(includeOnly))));
-      }
-
-      if (excludeQueryFile != null && !excludeQueryFile.equals("")) {
-        // Exclude specified query files, comma separated
-        for (String qFile : TEST_SPLITTER.split(excludeQueryFile)) {
-          if (null != queryDir) {
-            testFiles.remove(new File(queryDir, qFile));
-          } else {
-            testFiles.remove(new File(qFile));
-          }
-        }
-      }
-
-      hiveRootDir = new File(hiveRootDirectory);
-      if (!hiveRootDir.exists()) {
-        throw new BuildException("Hive Root Directory "
-            + hiveRootDir.getCanonicalPath() + " does not exist");
-      }
-
-      qFiles = new ArrayList<File>(testFiles);
-      Collections.sort(qFiles);
-      for (File qFile : qFiles) {
-        qFilesMap.put(qFile.getName(), relativePath(hiveRootDir, qFile));
-      }
-
-      // Make sure the output directory exists, if it doesn't then create it.
-      outDir = new File(outputDirectory);
-      if (!outDir.exists()) {
-        outDir.mkdirs();
-      }
-
-      logDir = new File(logDirectory);
-      if (!logDir.exists()) {
-        throw new BuildException("Log Directory " + logDir.getCanonicalPath() + " does not exist");
-      }
-
-      if (resultsDirectory != null) {
-        resultsDir = new File(resultsDirectory);
-        if (!resultsDir.exists()) {
-          throw new BuildException("Results Directory "
-              + resultsDir.getCanonicalPath() + " does not exist");
-        }
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-      throw new BuildException(e);
-    }
-
-    VelocityEngine ve = new VelocityEngine();
-
-    try {
-      ve.setProperty(RuntimeConstants.FILE_RESOURCE_LOADER_PATH, getTemplatePath());
-      if (logFile != null) {
-        File lf = new File(logFile);
-        if (lf.exists()) {
-          System.out.println("Log file already exists: " + lf.getCanonicalPath());
-          if (!lf.delete()) {
-            System.out.println("Could not delete log file " + lf.getCanonicalPath());
-            logFile = createAlternativeFile(lf);
-          }
-        }
-
-        ve.setProperty(RuntimeConstants.RUNTIME_LOG, logFile);
-      }
-
-      ve.init();
-      Template t = ve.getTemplate(template);
-
-      if (clusterMode == null) {
-        clusterMode = "";
-      }
-      if (hadoopVersion == null) {
-        hadoopVersion = "";
-      }
-
-      File qFileNames = new File(outputDirectory, className + "QFileNames.txt");
-      String qFileNamesFile = qFileNames.toURI().getPath();
-
-      if (qFileNames.exists()) {
-        System.out.println("Query file names containing file already exists: " + qFileNamesFile);
-        if (!qFileNames.delete()) {
-          System.out.println("Could not delete query file names containing file " +
-            qFileNames.getCanonicalPath());
-          qFileNamesFile = createAlternativeFile(qFileNames);
-        } else if (!qFileNames.createNewFile()) {
-          System.out.println("Could not create query file names containing file " +
-            qFileNamesFile);
-          qFileNamesFile = createAlternativeFile(qFileNames);
-        }
-      }
-      FileWriter fw = new FileWriter(qFileNames.getCanonicalFile());
-      BufferedWriter bw = new BufferedWriter(fw);
-
-      for (File qFile: qFiles) {
-        bw.write(qFile.getName());
-        bw.newLine();
-      }
-      bw.close();
-
-      // For each of the qFiles generate the test
-      System.out.println("hiveRootDir = " + hiveRootDir);
-      VelocityContext ctx = new VelocityContext();
-      ctx.put("className", className);
-      ctx.put("hiveRootDir", escapePath(hiveRootDir.getCanonicalPath()));
-      System.out.println("hiveRootDir = " + hiveRootDir);
-      System.out.println("queryDir = " + queryDir);
-      String strQueryDir = relativePath(hiveRootDir, queryDir);
-      System.out.println("queryDir = " + strQueryDir);
-      ctx.put("queryDir", strQueryDir);
-      ctx.put("qfiles", qFiles);
-      ctx.put("qFileNamesFile", qFileNamesFile);
-      ctx.put("qfilesMap", qFilesMap);
-      if (resultsDir != null) {
-        ctx.put("resultsDir", relativePath(hiveRootDir, resultsDir));
-      }
-      ctx.put("logDir", relativePath(hiveRootDir, logDir));
-      ctx.put("clusterMode", clusterMode);
-      if (hiveConfDir == null || hiveConfDir.isEmpty()) {
-        ctx.put("hiveConfDir", "");
-      } else {
-        System.out.println("hiveConfDir = " + hiveConfDir);
-        hiveConfDir = relativePath(hiveRootDir, new File(hiveConfDir));
-        System.out.println("hiveConfDir = " + hiveConfDir);
-        if (!(new File(hiveRootDir, hiveConfDir)).isDirectory()) {
-          throw new BuildException("hiveConfDir is not dir " + new File(hiveRootDir, hiveConfDir));
-        }
-        ctx.put("hiveConfDir", hiveConfDir);
-      }
-      ctx.put("hadoopVersion", hadoopVersion);
-      ctx.put("initScript", initScript);
-      ctx.put("cleanupScript", cleanupScript);
-      ctx.put("useHBaseMetastore", useHBaseMetastore);
-
-      File outFile = new File(outDir, className + ".java");
-      FileWriter writer = new FileWriter(outFile);
-      t.merge(ctx, writer);
-      writer.close();
-
-      System.out.println("Generated " + outFile.getCanonicalPath() + " from template " + template);
-    } catch(BuildException e) {
-      throw e;
-    } catch(MethodInvocationException e) {
-      throw new BuildException("Exception thrown by '" + e.getReferenceName() + "." +
-                               e.getMethodName() +"'",
-                               e.getWrappedThrowable());
-    } catch(ParseErrorException e) {
-      throw new BuildException("Velocity syntax error", e);
-    } catch(ResourceNotFoundException e) {
-      throw new BuildException("Resource not found", e);
-    } catch(Exception e) {
-      e.printStackTrace();
-      throw new BuildException("Generation failed", e);
-    }
-  }
-
-  private String relativePath(File hiveRootDir, File file) {
-    return escapePath(hiveRootDir.toURI().relativize(file.toURI()).getPath());
-  }
-
-  private static String escapePath(String path) {
-    if (System.getProperty("os.name").toLowerCase().startsWith("win")) {
-      // Escape the backward slash in CanonicalPath if the unit test runs on windows
-      // e.g. dir.getCanonicalPath() gets the absolute path of local
-      // directory. When we embed it directly in the generated java class it results
-      // in compiler error in windows. Reason : the canonical path contains backward
-      // slashes "C:\temp\etc\" and it is not a valid string in Java
-      // unless we escape the backward slashes.
-      return path.replace("\\", "\\\\");
-    }
-    return path;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/ant/src/org/apache/hadoop/hive/ant/antlib.xml
----------------------------------------------------------------------
diff --git a/ant/src/org/apache/hadoop/hive/ant/antlib.xml b/ant/src/org/apache/hadoop/hive/ant/antlib.xml
index 8f66348..1e42f0c 100644
--- a/ant/src/org/apache/hadoop/hive/ant/antlib.xml
+++ b/ant/src/org/apache/hadoop/hive/ant/antlib.xml
@@ -19,8 +19,6 @@
 
 
 <antlib>
-  <taskdef name="qtestgen"
-           classname="org.apache.hadoop.hive.ant.QTestGenTask" />
   <taskdef name="getversionpref"
            classname="org.apache.hadoop.hive.ant.GetVersionPref" />
 </antlib>

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/hbase-handler/src/test/templates/TestHBaseCliDriver.vm
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/templates/TestHBaseCliDriver.vm b/hbase-handler/src/test/templates/TestHBaseCliDriver.vm
deleted file mode 100644
index f513f03..0000000
--- a/hbase-handler/src/test/templates/TestHBaseCliDriver.vm
+++ /dev/null
@@ -1,144 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.cli;
-
-import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
-import org.apache.hadoop.hive.hbase.HBaseQTestUtil;
-import org.apache.hadoop.hive.hbase.HBaseTestSetup;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class $className {
-
-  private static final String HIVE_ROOT = HBaseQTestUtil.ensurePathEndsInSlash(System.getProperty("hive.root"));
-  private static HBaseQTestUtil qt;
-  private static HBaseTestSetup setup = new HBaseTestSetup();
-
-  @BeforeClass
-  public static void systemSetUp() {
-        MiniClusterType miniMR = MiniClusterType.valueForString("$clusterMode");
-        String initScript = "$initScript";
-        String cleanupScript = "$cleanupScript";
-
-        try {
-          qt = new HBaseQTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR,
-          setup, initScript, cleanupScript);
-          qt.cleanUp(null);
-          qt.createSources(null);
-
-        } catch (Exception e) {
-          System.err.println("Exception: " + e.getMessage());
-          e.printStackTrace();
-          System.err.flush();
-          fail("Unexpected exception in static initialization: "+e.getMessage());
-        }
-
-  }
-
-  @Before
-  public void setUp() {
-    try {
-      qt.clearTestSideEffects();
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in setup");
-    }
-  }
-  @After
-  public void tearDown() {
-    try {
-      qt.clearPostTestEffects();
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in tearDown");
-    }
-  }
-
-  @AfterClass
-  public static void shutdown() throws Exception {
-    try {
-      qt.shutdown();
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in shutdown");
-    }
-  }
-
-  @AfterClass
-  public static void closeHBaseConnections() throws Exception {
-    setup.tearDown();
-  }
-
-#foreach ($qf in $qfiles)
-  #set ($fname = $qf.getName())
-  #set ($eidx = $fname.indexOf('.'))
-  #set ($tname = $fname.substring(0, $eidx))
-  #set ($fpath = $qfilesMap.get($fname))
-  @Test
-  public void testCliDriver_$tname() throws Exception {
-    runTest("$tname", "$fname", (HIVE_ROOT + "$fpath"));
-  }
-
-#end
-
-  private void runTest(String tname, String fname, String fpath) throws Exception {
-    long startTime = System.currentTimeMillis();
-    try {
-      System.err.println("Begin query: " + fname);
-
-      qt.addFile(fpath);
-
-      if (qt.shouldBeSkipped(fname)) {
-        System.err.println("Test " + fname + " skipped");
-        return;
-      }
-
-      qt.cliInit(fname, false);
-
-      int ecode = qt.executeClient(fname);
-      if (ecode != 0) {
-        qt.failed(ecode, fname, null);
-      }
-
-      ecode = qt.checkCliDriverResults(fname);
-      if (ecode != 0) {
-        qt.failedDiff(ecode, fname, null);
-      }
-
-    } catch (Throwable e) {
-      qt.failed(e, fname, null);
-    }
-
-    long elapsedTime = System.currentTimeMillis() - startTime;
-    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
-    assertTrue("Test passed", true);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/hbase-handler/src/test/templates/TestHBaseNegativeCliDriver.vm
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/templates/TestHBaseNegativeCliDriver.vm b/hbase-handler/src/test/templates/TestHBaseNegativeCliDriver.vm
deleted file mode 100644
index 043bd87..0000000
--- a/hbase-handler/src/test/templates/TestHBaseNegativeCliDriver.vm
+++ /dev/null
@@ -1,119 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.cli;
-
-import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
-import org.apache.hadoop.hive.hbase.HBaseQTestUtil;
-import org.apache.hadoop.hive.hbase.HBaseTestSetup;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class $className {
-
-  private static final String HIVE_ROOT = HBaseQTestUtil.ensurePathEndsInSlash(System.getProperty("hive.root"));
-  private HBaseQTestUtil qt;
-  private static HBaseTestSetup setup = new HBaseTestSetup();
-
-  @Before
-  public void setUp() {
-
-    MiniClusterType miniMR = MiniClusterType.valueForString("$clusterMode");
-    String initScript = "$initScript";
-    String cleanupScript = "$cleanupScript";
-
-    try {
-      qt = new HBaseQTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR,
-      setup, initScript, cleanupScript);
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in setup");
-    }
-  }
-
-  @After
-  public void tearDown() {
-    try {
-      qt.shutdown();
-    } catch (Exception e) {
-      System.err.println("Exception: " + e.getMessage());
-      e.printStackTrace();
-      System.err.flush();
-      fail("Unexpected exception in tearDown");
-    }
-  }
-
-  @AfterClass
-  public static void closeHBaseConnections() throws Exception {
-    setup.tearDown();
-  }
-
-#foreach ($qf in $qfiles)
-  #set ($fname = $qf.getName())
-  #set ($eidx = $fname.indexOf('.'))
-  #set ($tname = $fname.substring(0, $eidx))
-  #set ($fpath = $qfilesMap.get($fname))
-  @Test
-  public void testCliDriver_$tname() throws Exception {
-    runTest("$tname", "$fname", (HIVE_ROOT + "$fpath"));
-  }
-
-#end
-
-  private void runTest(String tname, String fname, String fpath) throws Exception {
-    long startTime = System.currentTimeMillis();
-    try {
-      System.err.println("Begin query: " + fname);
-
-      qt.addFile(fpath);
-
-      if (qt.shouldBeSkipped(fname)) {
-        System.err.println("Test " + fname + " skipped");
-        return;
-      }
-
-      qt.cliInit(fname);
-      qt.clearTestSideEffects();
-      int ecode = qt.executeClient(fname);
-      if (ecode == 0) {
-        qt.failed(fname, null);
-      }
-
-      ecode = qt.checkCliDriverResults(fname);
-      if (ecode != 0) {
-        qt.failedDiff(ecode, fname, null);
-      }
-      qt.clearPostTestEffects();
-
-    } catch (Throwable e) {
-      qt.failed(e, fname, null);
-    }
-
-    long elapsedTime = System.currentTimeMillis() - startTime;
-    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
-    assertTrue("Test passed", true);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest-accumulo/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest-accumulo/pom.xml b/itests/qtest-accumulo/pom.xml
index 339c599..a4a5c75 100644
--- a/itests/qtest-accumulo/pom.xml
+++ b/itests/qtest-accumulo/pom.xml
@@ -431,15 +431,13 @@
                 <echo message="${test.classpath}"/>
                 <taskdef resource="net/sf/antcontrib/antcontrib.properties"
                   classpathref="maven.plugin.classpath" />
-                <taskdef name="qtestgen" classname="org.apache.hadoop.hive.ant.QTestGenTask"
-                  classpath="${test.classpath}" />
 
                 <!-- Accumulo Positive -->
                 <if>
                   <equals arg1="${skip.accumulo.tests}" arg2="false" />
                     <then>
                       <mkdir dir="${project.build.directory}/qfile-results/accumulo-handler/positive/" />
-                      <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
+<!-- HIVE-14444       <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}/accumulo-handler/src/test/templates/" template="TestAccumuloCliDriver.vm"
                       queryDirectory="${basedir}/${hive.path.to.root}/accumulo-handler/src/test/queries/positive/"
@@ -451,6 +449,7 @@
                       logDirectory="${project.build.directory}/qfile-results/accumulo-handler/positive/"
                       initScript="q_test_init.sql"
                       cleanupScript="q_test_cleanup.sql"/>
+-->
                     </then>
                 </if>
               </target>

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest-spark/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest-spark/pom.xml b/itests/qtest-spark/pom.xml
index 3bc9e24..1e6c3a2 100644
--- a/itests/qtest-spark/pom.xml
+++ b/itests/qtest-spark/pom.xml
@@ -374,65 +374,11 @@
                 <echo message="${test.classpath}"/>
                 <taskdef resource="net/sf/antcontrib/antcontrib.properties"
                   classpathref="maven.plugin.classpath" />
-                <taskdef name="qtestgen" classname="org.apache.hadoop.hive.ant.QTestGenTask"
-                  classpath="${test.classpath}" />
                 <mkdir dir="${project.build.directory}/qfile-results/clientpositive/spark" />
                 <mkdir dir="${project.build.directory}/qfile-results/clientpositive/miniSparkOnYarn" />
                 <mkdir dir="${project.build.directory}/qfile-results/clientnegative/spark" />
                 <mkdir dir="${project.build.directory}/qfile-results/clientnegative/miniSparkOnYarn" />
 
-                <!-- Cli with Spark standalone mode -->
-                <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}"
-                  queryFileRegex="${qfile_regex}"
-                  clusterMode="spark"
-                  includeQueryFile="${spark.query.files}"
-                  runDisabled="${run_disabled}"
-                  hiveConfDir="${basedir}/${hive.path.to.root}/data/conf/spark/standalone"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientpositive/spark"
-                  className="TestSparkCliDriver"
-                  logFile="${project.build.directory}/testsparkclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/clientpositive/spark"
-                  initScript="q_test_init.sql"
-                  cleanupScript="q_test_cleanup.sql"/>
-
-                <!-- Cli with Spark on YARN -->
-                <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}"
-                  queryFileRegex="${qfile_regex}"
-                  clusterMode="miniSparkOnYarn"
-                  includeQueryFile="${miniSparkOnYarn.query.files}"
-                  runDisabled="${run_disabled}"
-                  hiveConfDir="${basedir}/${hive.path.to.root}/data/conf/spark/yarn-client"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientpositive/spark"
-                  className="TestMiniSparkOnYarnCliDriver"
-                  logFile="${project.build.directory}/testminisparkonyarnclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/clientpositive/spark"
-                  initScript="q_test_init.sql"
-                  cleanupScript="q_test_cleanup.sql"/>
-
-                <!-- Negative Cli with Spark standalone mode -->
-                <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="TestNegativeCliDriver.vm"
-                  queryDirectory="${basedir}/${hive.path.to.root}/ql/src/test/queries/clientnegative/"
-                  queryFile="${qfile}"
-                  queryFileRegex="${qfile_regex}"
-                  clusterMode="spark"
-                  includeQueryFile="${spark.query.negative.files}"
-                  runDisabled="${run_disabled}"
-                  hiveConfDir="${basedir}/${hive.path.to.root}/data/conf/spark/standalone"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientnegative/spark" className="TestSparkNegativeCliDriver"
-                  logFile="${project.build.directory}/testsparknegativeclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/clientnegative/spark"
-                  initScript="q_test_init.sql"
-                  cleanupScript="q_test_cleanup.sql"/>
 
               </target>
             </configuration>

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestMiniSparkOnYarnCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestMiniSparkOnYarnCliDriver.java b/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestMiniSparkOnYarnCliDriver.java
new file mode 100644
index 0000000..e84bfce
--- /dev/null
+++ b/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestMiniSparkOnYarnCliDriver.java
@@ -0,0 +1,45 @@
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestMiniSparkOnYarnCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.SparkOnYarnCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestMiniSparkOnYarnCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkCliDriver.java b/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkCliDriver.java
new file mode 100644
index 0000000..2c8cbee
--- /dev/null
+++ b/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkCliDriver.java
@@ -0,0 +1,45 @@
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestSparkCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.SparkCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestSparkCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkNegativeCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkNegativeCliDriver.java b/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkNegativeCliDriver.java
new file mode 100644
index 0000000..2db83f4
--- /dev/null
+++ b/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkNegativeCliDriver.java
@@ -0,0 +1,45 @@
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestSparkNegativeCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.SparkNegativeCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestSparkNegativeCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml
index 17968e6..ed44bb8 100644
--- a/itests/qtest/pom.xml
+++ b/itests/qtest/pom.xml
@@ -30,7 +30,7 @@
 
   <properties>
     <hive.path.to.root>../..</hive.path.to.root>
-    <initScript>q_test_init.sql</initScript>
+<!--    <initScript></initScript>-->
     <qfile></qfile>
     <qfile_regex></qfile_regex>
     <run_disabled>false</run_disabled>
@@ -393,8 +393,6 @@
                 <echo message="${test.classpath}"/>
                 <taskdef resource="net/sf/antcontrib/antcontrib.properties"
                   classpathref="maven.plugin.classpath" />
-                <taskdef name="qtestgen" classname="org.apache.hadoop.hive.ant.QTestGenTask"
-                  classpath="${test.classpath}" />
                 <mkdir dir="${project.build.directory}/qfile-results/clientpositive/" />
                 <mkdir dir="${project.build.directory}/qfile-results/clientpositive/perf" /> 
                 <mkdir dir="${project.build.directory}/qfile-results/clientnegative/" />
@@ -410,271 +408,8 @@
                 <mkdir dir="${project.build.directory}/qfile-results/contribclientpositive"/>
                 <mkdir dir="${project.build.directory}/qfile-results/contribclientnegative"/>
 
-                <!-- Negative Parse -->
-                <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
-                  outputDirectory="${project.build.directory}/generated-test-sources/java/org/apache/hadoop/hive/ql/parse/"
-                  templatePath="${basedir}/${hive.path.to.root}/ql/src/test/templates/" template="TestParseNegative.vm"
-                  queryDirectory="${basedir}/${hive.path.to.root}/ql/src/test/queries/negative/"
-                  queryFile="${qfile}"
-                  queryFileRegex="${qfile_regex}"
-                  runDisabled="${run_disabled}"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/compiler/errors/" className="TestParseNegative"
-                  logFile="${project.build.directory}/testparseneggen.log"
-                  hadoopVersion="${hadoop.version}"
-                  logDirectory="${project.build.directory}/qfile-results/negative/"
-                  initScript="${initScript}"
-                  cleanupScript="q_test_cleanup.sql"/>
 
-                <!-- Cli -->
-                <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="${minillap.query.files},${minimr.query.files},${minitez.query.files},${encrypted.query.files},${spark.only.query.files},${disabled.query.files}"
-                  queryFileRegex="${qfile_regex}"
-                  clusterMode="${clustermode}"
-                  runDisabled="${run_disabled}"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientpositive/" className="TestCliDriver"
-                  logFile="${project.build.directory}/testclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/clientpositive/"
-                  hadoopVersion="${hadoop.version}"
-                  initScript="${initScript}"
-                  cleanupScript="q_test_cleanup.sql"/>
 
-		<!-- PerfCli -->
-                <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="TestPerfCliDriver.vm"
-                  queryDirectory="${basedir}/${hive.path.to.root}/ql/src/test/queries/clientpositive/perf"
-                  queryFile="${qfile}"
-                  excludeQueryFile="${minimr.query.files},${minitez.query.files},${encrypted.query.files}"
-                  queryFileRegex="${qfile_regex}"
-                  clusterMode="tez"
-                  runDisabled="${run_disabled}"
-                  hiveConfDir="${basedir}/${hive.path.to.root}/data/conf/perf-reg"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientpositive/perf" className="TestPerfCliDriver"
-                  logFile="${project.build.directory}/testperfclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/clientpositive"
-                  hadoopVersion="${hadoop.version}"
-	          initScript="q_perf_test_init.sql"
-                  cleanupScript="q_perf_test_cleanup.sql"
-                  useHBaseMetastore="false"/>
-
-                <!-- Negative Cli -->
-                <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="TestNegativeCliDriver.vm"
-                  queryDirectory="${basedir}/${hive.path.to.root}/ql/src/test/queries/clientnegative/"
-                  queryFile="${qfile}"
-                  excludeQueryFile="${minimr.query.negative.files},authorization_uri_import.q"
-                  queryFileRegex="${qfile_regex}"
-                  clusterMode="${clustermode}"
-                  runDisabled="${run_disabled}"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientnegative/" className="TestNegativeCliDriver"
-                  logFile="${project.build.directory}/testnegativeclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/clientnegative/"
-                  hadoopVersion="${hadoop.version}"
-                  initScript="${initScript}"
-                  cleanupScript="q_test_cleanup.sql"/>
-
-                <!-- Compare Cli -->
-                <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="TestCompareCliDriver.vm"
-                  queryDirectory="${basedir}/${hive.path.to.root}/ql/src/test/queries/clientcompare/"
-                  queryFile="${qfile}"
-                  queryFileRegex="${qfile_regex}"
-                  clusterMode="${clustermode}"
-                  runDisabled="${run_disabled}"
-                  className="TestCompareCliDriver"
-                  logFile="${project.build.directory}/testcompareclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/clientcompare/"
-                  hadoopVersion="${hadoop.version}"
-                  initScript="${initScript}"
-                  cleanupScript="q_test_cleanup.sql"/>
-
-                <!-- Minimr -->
-                <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}"
-                  includeQueryFile="${minimr.query.files}"
-                  queryFileRegex="${qfile_regex}"
-                  clusterMode="miniMR"
-                  runDisabled="${run_disabled}"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientpositive/" className="TestMinimrCliDriver"
-                  logFile="${project.build.directory}/testminimrclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/clientpositive/"
-                  hadoopVersion="${hadoop.version}"
-                  initScript="${initScript}"
-                  cleanupScript="q_test_cleanup.sql"/>
-
-                <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}"
-                  includeQueryFile="${minitez.query.files},${minitez.query.files.shared}"
-                  queryFileRegex="${qfile_regex}"
-                  clusterMode="tez"
-                  runDisabled="${run_disabled}"
-                  hiveConfDir="${basedir}/${hive.path.to.root}/data/conf/tez"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientpositive/tez"
-                  className="TestMiniTezCliDriver"
-                  logFile="${project.build.directory}/testminitezclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/clientpositive/"
-                  hadoopVersion="${hadoop.version}"
-                  initScript="${initScript}"
-                  cleanupScript="q_test_cleanup.sql"
-                  useHBaseMetastore="true"/>
-
-                <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}"
-                  includeQueryFile="${minillap.query.files},${minillap.shared.query.files}"
-                  queryFileRegex="${qfile_regex}"
-                  clusterMode="llap"
-                  runDisabled="${run_disabled}"
-                  hiveConfDir="${basedir}/${hive.path.to.root}/data/conf/llap"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientpositive/llap"
-                  className="TestMiniLlapCliDriver"
-                  logFile="${project.build.directory}/testminitezclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/clientpositive/"
-                  hadoopVersion="${hadoop.version}"
-		  initScript="${initScript}"
-                  cleanupScript="q_test_cleanup.sql"/>
-
-                <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}"
-                  includeQueryFile="${encrypted.query.files}"
-                  queryFileRegex="${qfile_regex}"
-                  clusterMode="encrypted"
-                  runDisabled="${run_disabled}"
-                  hiveConfDir="${basedir}/${hive.path.to.root}/data/conf"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientpositive/encrypted"
-                  className="TestEncryptedHDFSCliDriver"
-                  logFile="${project.build.directory}/testencryptedhdfsclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/clientpositive/"
-                  hadoopVersion="${hadoop.version}"
-                  initScript="q_test_init_for_encryption.sql"
-                  cleanupScript="q_test_cleanup_for_encryption.sql"/>
-
-
-                <!-- Negative Minimr -->
-                <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="TestNegativeCliDriver.vm"
-                  queryDirectory="${basedir}/${hive.path.to.root}/ql/src/test/queries/clientnegative/"
-                  queryFile="${qfile}"
-                  includeQueryFile="${minimr.query.negative.files}"
-                  queryFileRegex="${qfile_regex}"
-                  clusterMode="miniMR"
-                  runDisabled="${run_disabled}"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientnegative/" className="TestNegativeMinimrCliDriver"
-                  logFile="${project.build.directory}/testnegativeminimrclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/clientnegative/"
-                  hadoopVersion="${hadoop.version}"
-                  initScript="${initScript}"
-                  cleanupScript="q_test_cleanup.sql"/>
-
-                <!-- HBase Positive -->
-                <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}/hbase-handler/src/test/templates/" template="TestHBaseCliDriver.vm"
-                  queryDirectory="${basedir}/${hive.path.to.root}/hbase-handler/src/test/queries/positive/"
-                  queryFile="${qfile}"
-                  runDisabled="${run_disabled}"
-                  clusterMode="${clustermode}"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/hbase-handler/src/test/results/positive/" className="TestHBaseCliDriver"
-                  logFile="${project.build.directory}/testhbaseclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/hbase-handler/positive/"
-                  initScript="${initScript}"
-                  cleanupScript="q_test_cleanup.sql"/>
-
-                <!-- HBase Minimr -->
-                <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}/hbase-handler/src/test/templates/" template="TestHBaseCliDriver.vm"
-                  queryDirectory="${basedir}/${hive.path.to.root}/hbase-handler/src/test/queries/positive/"
-                  queryFile="hbase_bulk.m"
-                  runDisabled="${run_disabled}"
-                  clusterMode="miniMR"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/hbase-handler/src/test/results/positive/" className="TestHBaseMinimrCliDriver"
-                  logFile="${project.build.directory}/testhbaseminimrclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/hbase-handler/minimrpositive/"
-                  initScript="${initScript}"
-                  cleanupScript="q_test_cleanup.sql"/>
-
-                <!-- HBase Negative -->
-                <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}/hbase-handler/src/test/templates/" template="TestHBaseNegativeCliDriver.vm"
-                  queryDirectory="${basedir}/${hive.path.to.root}/hbase-handler/src/test/queries/negative/"
-                  queryFile="${qfile}"
-                  runDisabled="${run_disabled}"
-                  clusterMode="${clustermode}"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/hbase-handler/src/test/results/negative/" className="TestHBaseNegativeCliDriver"
-                  logFile="${project.build.directory}/testhbasenegativeclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/hbase-handler/negative"
-                  initScript="${initScript}"
-                  cleanupScript="q_test_cleanup.sql"/>
-
-                <!-- Beeline -->
-                <if>
-                  <equals arg1="${execute.beeline.tests}" arg2="true" />
-                    <then>
-                      <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
-                        outputDirectory="${project.build.directory}/generated-test-sources/java/org/apache/hive/beeline/util/"
-                        templatePath="${ql.test.template.dir}" template="TestBeeLineDriver.vm"
-                        queryDirectory="${basedir}/${hive.path.to.root}/ql/src/test/queries/clientpositive/"
-                        queryFile="${qfile}"
-                        excludeQueryFile="${beeline.positive.exclude}"
-                        queryFileRegex="${qfile_regex}"
-                        clusterMode="${clustermode}"
-                        runDisabled="${run_disabled}"
-                        resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientpositive/" className="TestBeeLineDriver"
-                        logFile="${project.build.directory}/testbeelinedrivergen.log"
-                        logDirectory="${project.build.directory}/qfile-results/beelinepositive/"
-                        hadoopVersion="${hadoop.version}"/>
-                    </then>
-                </if>
-
-
-
-                <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}/contrib/src/test/queries/clientpositive"
-                  queryFile="${qfile}"
-                  queryFileRegex="${qfile_regex}"
-                  runDisabled="${run_disabled}"
-                  clusterMode="${clustermode}"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/contrib/src/test/results/clientpositive/" className="TestContribCliDriver"
-                  logFile="${project.build.directory}/testcontribclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/contribclientpositive"
-                  hadoopVersion="${hadoop.version}"
-                  initScript="${initScript}"
-                  cleanupScript="q_test_cleanup.sql"/>
-
-                <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="TestNegativeCliDriver.vm"
-                  queryDirectory="${basedir}/${hive.path.to.root}/contrib/src/test/queries/clientnegative"
-                  queryFile="${qfile}"
-                  queryFileRegex="${qfile_regex}"
-                  runDisabled="${run_disabled}"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/contrib/src/test/results/clientnegative/" className="TestContribNegativeCliDriver"
-                  logFile="${project.build.directory}/testcontribnegclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/contribclientnegative"
-                  initScript="${initScript}"
-                  cleanupScript="q_test_cleanup.sql"/>
 
               </target>
             </configuration>

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/ContribNegativeCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/ContribNegativeCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/ContribNegativeCliDriver.java
new file mode 100644
index 0000000..253cda3
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/ContribNegativeCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class ContribNegativeCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.ContribNegativeCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public ContribNegativeCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/DisabledTestBeeLineDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/DisabledTestBeeLineDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/DisabledTestBeeLineDriver.java
new file mode 100644
index 0000000..cb276e6
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/DisabledTestBeeLineDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class DisabledTestBeeLineDriver {
+
+  static CliAdapter adapter = new CliConfigs.BeeLineConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public DisabledTestBeeLineDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/DummyCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/DummyCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/DummyCliDriver.java
new file mode 100644
index 0000000..965d1dc
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/DummyCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class DummyCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.DummyConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public DummyCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestCliDriver.java
new file mode 100644
index 0000000..c4c4f41
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.CliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestCompareCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestCompareCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestCompareCliDriver.java
new file mode 100644
index 0000000..944cd32
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestCompareCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestCompareCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.CompareCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestCompareCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestContribCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestContribCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestContribCliDriver.java
new file mode 100644
index 0000000..54596f9
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestContribCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestContribCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.ContribCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestContribCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestContribNegativeCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestContribNegativeCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestContribNegativeCliDriver.java
new file mode 100644
index 0000000..1b39ee7
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestContribNegativeCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestContribNegativeCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.ContribNegativeCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestContribNegativeCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestEncryptedHDFSCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestEncryptedHDFSCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestEncryptedHDFSCliDriver.java
new file mode 100644
index 0000000..8c6807e
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestEncryptedHDFSCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestEncryptedHDFSCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.EncryptedHDFSCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestEncryptedHDFSCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseCliDriver.java
new file mode 100644
index 0000000..7b6f76a
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestHBaseCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.HBaseCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestHBaseCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseMinimrCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseMinimrCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseMinimrCliDriver.java
new file mode 100644
index 0000000..934af16
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseMinimrCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestHBaseMinimrCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.HBaseMinimrCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestHBaseMinimrCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}


[2/3] hive git commit: HIVE-14444 Upgrade qtest execution framework to junit4 - migrate most of them (Zoltan Haindrich via Ashutosh Chauhan)

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseNegativeCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseNegativeCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseNegativeCliDriver.java
new file mode 100644
index 0000000..88d626c
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestHBaseNegativeCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestHBaseNegativeCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.HBaseNegativeCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestHBaseNegativeCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniLlapCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniLlapCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniLlapCliDriver.java
new file mode 100644
index 0000000..ad525fe
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniLlapCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestMiniLlapCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.MiniLlapCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestMiniLlapCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniTezCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniTezCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniTezCliDriver.java
new file mode 100644
index 0000000..c23b0b3
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniTezCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestMiniTezCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.MiniTezCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestMiniTezCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMinimrCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMinimrCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMinimrCliDriver.java
new file mode 100644
index 0000000..96a9e8f
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMinimrCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestMinimrCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.MinimrCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestMinimrCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestNegativeCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestNegativeCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestNegativeCliDriver.java
new file mode 100644
index 0000000..1040228
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestNegativeCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestNegativeCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.NegativeCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestNegativeCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestNegativeMinimrCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestNegativeMinimrCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestNegativeMinimrCliDriver.java
new file mode 100644
index 0000000..f7e2caa
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestNegativeMinimrCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestNegativeMinimrCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.NegativeMinimrCli().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestNegativeMinimrCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestPerfCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestPerfCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestPerfCliDriver.java
new file mode 100644
index 0000000..4df4eeb
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestPerfCliDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestPerfCliDriver {
+
+  static CliAdapter adapter = new CliConfigs.PerfCliConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestPerfCliDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/qtest/src/test/java/org/apache/hadoop/hive/ql/parse/TestParseNegativeDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/ql/parse/TestParseNegativeDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/ql/parse/TestParseNegativeDriver.java
new file mode 100644
index 0000000..4c1224f
--- /dev/null
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/ql/parse/TestParseNegativeDriver.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.parse;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestParseNegativeDriver {
+
+  static CliAdapter adapter = new CliConfigs.ParseNegativeConfig().getCliAdapter();
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() throws Exception {
+    return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public TestParseNegativeDriver(String name, File qfile) {
+    this.name = name;
+    this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+    adapter.runTest(name, qfile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java
new file mode 100644
index 0000000..efbd465
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCliConfig.java
@@ -0,0 +1,417 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli.control;
+
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.reflect.Constructor;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hive.ql.QTestUtil;
+import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Sets;
+
+public abstract class AbstractCliConfig {
+
+  public static final String HIVE_ROOT = getHiveRoot();
+
+  public static enum MetastoreType {
+    sql, hbase
+  };
+
+  private MetastoreType metastoreType = MetastoreType.sql;
+  private String queryFile;
+  private String queryFileRegex;
+  private String queryDirectory;
+  // pending change to boolean
+  private String runDisabled;
+  // FIXME: file paths in strings should be changed to either File or Path ... anything but String
+  private String resultsDirectory;
+  private Set<String> excludedQueryFileNames = new LinkedHashSet<>();
+  private String hadoopVersion;
+  private String logDirectory;
+  // these should have viable defaults
+  private String cleanupScript;
+  private String initScript;
+  private String hiveConfDir;
+  private MiniClusterType clusterType;
+
+  // FIXME: null value is treated differently on the other end..when those filter will be
+  // moved...this may change
+  private Set<String> includeQueryFileNames;
+  private Class<? extends CliAdapter> cliAdapter;
+
+  public AbstractCliConfig(Class<? extends CliAdapter> adapter) {
+    cliAdapter=adapter;
+    clusterType = MiniClusterType.none;
+    queryFile = getSysPropValue("qfile");
+    queryFileRegex = getSysPropValue("qfile_regex");
+    runDisabled = getSysPropValue("run_disabled");
+  }
+
+  private static String getHiveRoot() {
+    List<String> candidateSiblings = new ArrayList<>();
+    if (System.getProperty("hive.root") != null) {
+      try {
+        candidateSiblings.add(new File(System.getProperty("hive.root")).getCanonicalPath());
+      } catch (IOException e) {
+        throw new RuntimeException("error getting hive.root",e);
+      }
+    }
+    candidateSiblings.add(new File(".").getAbsolutePath());
+
+    for (String string : candidateSiblings) {
+      File curr = new File(string);
+      do {
+        Set<String> lls = Sets.newHashSet(curr.list());
+        if (lls.contains("itests") && lls.contains("ql") && lls.contains("metastore")) {
+          System.out.println("detected hiveRoot: " + curr);
+          return QTestUtil.ensurePathEndsInSlash(curr.getAbsolutePath());
+        }
+        curr = curr.getParentFile();
+      } while (curr != null);
+    }
+    throw new RuntimeException("unable to find hiveRoot");
+  }
+
+  protected void setQueryDir(String dir) {
+    queryDirectory = getAbsolutePath(dir);
+  }
+
+  @Deprecated
+  public void overrideUserQueryFile(String q) {
+    queryFile = q;
+  }
+
+  public void includesFrom(URL resource, String key) {
+    try (InputStream is = resource.openStream()) {
+      Properties props = new Properties();
+      props.load(is);
+      String fileNames = getSysPropValue(key);
+      if (fileNames == null) {
+        fileNames = props.getProperty(key);
+      }
+      if (fileNames != null) {
+        for (String qFile : TEST_SPLITTER.split(fileNames)) {
+          includeQuery(qFile);
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("error processing:" + resource, e);
+    }
+  }
+
+  protected void includeQuery(String qFile) {
+    if (includeQueryFileNames == null) {
+      includeQueryFileNames = new HashSet<>();
+    }
+    includeQueryFileNames.add(qFile);
+  }
+
+  public void excludesFrom(URL resource, String key) {
+    try (InputStream is = resource.openStream()) {
+      Properties props = new Properties();
+      props.load(is);
+
+      String fileNames = getSysPropValue(key);
+      if (fileNames == null) {
+        fileNames = props.getProperty(key);
+      }
+      if (fileNames != null) {
+        for (String qFile : TEST_SPLITTER.split(fileNames)) {
+          excludeQuery(qFile);
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("error processing:" + resource, e);
+    }
+  }
+
+  protected void excludeQuery(String qFile) {
+    excludedQueryFileNames.add(qFile);
+  }
+
+
+  private static final Splitter TEST_SPLITTER =
+      Splitter.onPattern("[, ]").trimResults().omitEmptyStrings();
+
+  public static class IncludeFilter implements FileFilter {
+
+    Set<String> includeOnly;
+
+    public IncludeFilter(Set<String> includeOnly) {
+      this.includeOnly = includeOnly;
+    }
+
+    @Override
+    public boolean accept(File fpath) {
+      return includeOnly == null || includeOnly.contains(fpath.getName());
+    }
+  }
+
+  public static class QFileFilter extends IncludeFilter {
+
+    public QFileFilter(Set<String> includeOnly) {
+      super(includeOnly);
+    }
+
+    @Override
+    public boolean accept(File fpath) {
+      if (!super.accept(fpath)) {
+        return false;
+      }
+      if (fpath.isDirectory() || !fpath.getName().endsWith(".q")) {
+        return false;
+      }
+      return true;
+    }
+  }
+
+  public static class DisabledQFileFilter extends IncludeFilter {
+    public DisabledQFileFilter(Set<String> includeOnly) {
+      super(includeOnly);
+    }
+
+    @Override
+    public boolean accept(File fpath) {
+      if (!super.accept(fpath)) {
+        return false;
+      }
+      return !fpath.isDirectory() && fpath.getName().endsWith(".q.disabled");
+    }
+  }
+
+  public static class QFileRegexFilter implements FileFilter {
+    Pattern filterPattern;
+
+    public QFileRegexFilter(String filter) {
+      filterPattern = Pattern.compile(filter);
+    }
+
+    @Override
+    public boolean accept(File filePath) {
+      if (filePath.isDirectory() || !filePath.getName().endsWith(".q")) {
+        return false;
+      }
+      String testName = StringUtils.chomp(filePath.getName(), ".q");
+      return filterPattern.matcher(testName).matches();
+    }
+  }
+
+  public Set<File> getQueryFiles() throws Exception {
+    prepareDirs();
+
+    Set<String> includeOnly = includeQueryFileNames;
+
+    // queryDirectory should not be null
+    File queryDir = new File(queryDirectory);
+
+    // dedup file list
+    Set<File> testFiles = new LinkedHashSet<>();
+    if (queryFile != null && !queryFile.equals("")) {
+      // The user may have passed a list of files - comma separated
+      for (String qFile : TEST_SPLITTER.split(queryFile)) {
+        if (null != queryDir) {
+          testFiles.add(new File(queryDir, qFile));
+        } else {
+          testFiles.add(new File(qFile));
+        }
+      }
+    } else if (queryFileRegex != null && !queryFileRegex.equals("")) {
+      for (String regex : TEST_SPLITTER.split(queryFileRegex)) {
+        testFiles.addAll(Arrays.asList(queryDir.listFiles(new QFileRegexFilter(regex))));
+      }
+    } else if (runDisabled != null && runDisabled.equals("true")) {
+      testFiles.addAll(Arrays.asList(queryDir.listFiles(new DisabledQFileFilter(includeOnly))));
+    } else {
+      testFiles.addAll(Arrays.asList(queryDir.listFiles(new QFileFilter(includeOnly))));
+    }
+
+    for (String qFileName : excludedQueryFileNames) {
+      testFiles.remove(new File(queryDir, qFileName));
+    }
+
+    return testFiles;
+  }
+
+  private void prepareDirs() throws Exception {
+    File hiveRootDir = new File(HIVE_ROOT);
+    if (!hiveRootDir.exists()) {
+      throw new RuntimeException(
+          "Hive Root Directory " + hiveRootDir.getCanonicalPath() + " does not exist");
+    }
+
+    File logDir = new File(logDirectory);
+    if (!logDir.exists()) {
+      FileUtils.forceMkdir(logDir);
+    }
+
+    File resultsDir = new File(resultsDirectory);
+    if (!resultsDir.exists()) {
+      FileUtils.forceMkdir(resultsDir);
+    }
+  }
+
+  public String getHadoopVersion() {
+    if (hadoopVersion == null) {
+      System.out.println("detecting hadoop.version from loaded libs");
+      try {
+        String hadoopPropsLoc = "/META-INF/maven/org.apache.hadoop/hadoop-hdfs/pom.properties";
+        URL hadoopPropsURL = getClass().getResource(hadoopPropsLoc);
+        if (hadoopPropsURL == null) {
+          throw new RuntimeException("failed to get hadoop properties: " + hadoopPropsLoc);
+        }
+        try (InputStream is = hadoopPropsURL.openStream()) {
+          Properties props = new Properties();
+          props.load(is);
+          hadoopVersion = props.getProperty("version");
+          if (hadoopVersion == null) {
+            throw new RuntimeException("version property not found");
+          }
+        } catch (IOException e) {
+          throw new RuntimeException("unable to extract hadoop.version from: " + hadoopPropsURL, e);
+        }
+      } catch (Exception e) {
+        throw new RuntimeException(
+            "can't get hadoop.version ; specify manually using hadoop.version property!");
+      }
+    }
+    return hadoopVersion;
+  }
+
+  protected void setHadoopVersion(String hadoopVersion) {
+    this.hadoopVersion = hadoopVersion;
+  }
+
+  public String getLogDir() {
+    return logDirectory;
+  }
+
+  protected void setLogDir(String logDirectory) {
+    this.logDirectory = getAbsolutePath(logDirectory);
+  }
+
+  public String getResultsDir() {
+    return resultsDirectory;
+  }
+
+  protected void setResultsDir(String resultsDir) {
+    resultsDirectory = getAbsolutePath(resultsDir);
+  }
+
+  public String getCleanupScript() {
+    return cleanupScript;
+  }
+
+  protected void setCleanupScript(String cleanupScript) {
+    this.cleanupScript = cleanupScript;
+  }
+
+  public String getInitScript() {
+    return initScript;
+  }
+
+  protected void setInitScript(String initScript) {
+    String initScriptPropValue = getSysPropValue("initScript");
+    if (initScriptPropValue != null) {
+      System.out.println("initScript override(by system property):" + initScriptPropValue);
+      this.initScript = initScriptPropValue;
+    } else {
+      this.initScript = initScript;
+    }
+  }
+
+  public String getHiveConfDir() {
+    return hiveConfDir;
+  }
+
+  protected void setHiveConfDir(String hiveConfDir) {
+    if (hiveConfDir.trim().isEmpty()) {
+      this.hiveConfDir = hiveConfDir;
+    } else {
+      this.hiveConfDir = getAbsolutePath(hiveConfDir);
+    }
+  }
+
+  public MiniClusterType getClusterType() {
+    return clusterType;
+  }
+
+  protected void setClusterType(MiniClusterType type) {
+    String modeStr = getSysPropValue("clustermode");
+    if (modeStr != null) {
+      // FIXME: this should be changeto valueOf ...
+      // that will also kill that fallback 'none' which is I think more like a problem than a
+      // feature ;)
+      clusterType = MiniClusterType.valueForString(modeStr);
+    } else {
+      clusterType = type;
+    }
+    if (clusterType == null) {
+      throw new RuntimeException("clustertype cant be null");
+    }
+  }
+
+  private String getSysPropValue(String propName) {
+    String propValue = System.getProperty(propName);
+    if (propValue == null || propValue.trim().length() == 0) {
+      return null;
+    }
+    System.out.println("property: " + propName + " used as override with val: " + propValue);
+    return propValue.trim();
+  }
+
+  public CliAdapter getCliAdapter() {
+    try {
+      Constructor<? extends CliAdapter> cz = cliAdapter.getConstructor(AbstractCliConfig.class);
+      return cz.newInstance(this);
+    } catch (Exception e) {
+      throw new RuntimeException("unable to build adapter", e);
+    }
+  }
+
+  protected void setMetastoreType(MetastoreType mt) {
+    metastoreType=mt;
+  }
+
+  public MetastoreType getMetastoreType() {
+    return metastoreType;
+  }
+
+  public String getQueryDirectory() {
+    return queryDirectory;
+  }
+
+  private String getAbsolutePath(String dir) {
+    return new File(new File(HIVE_ROOT), dir).getAbsolutePath();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliAdapter.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliAdapter.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliAdapter.java
new file mode 100644
index 0000000..b89d6e7
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliAdapter.java
@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli.control;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+
+/**
+ * This class adapts old vm test-executors to be executed in multiple instances
+ */
+public abstract class CliAdapter {
+
+  protected final AbstractCliConfig cliConfig;
+
+  public CliAdapter(AbstractCliConfig cliConfig) {
+    this.cliConfig = cliConfig;
+  }
+
+  public final List<Object[]> getParameters() throws Exception {
+    Set<File> f = cliConfig.getQueryFiles();
+    List<Object[]> ret = new ArrayList<>();
+
+    for (File file : f) {
+      String label = file.getName().replaceAll("\\.[^\\.]+$", "");
+      ret.add(new Object[] { label, file });
+    }
+    return ret;
+  }
+
+  abstract public void beforeClass() throws Exception;
+
+  // HIVE-14444 pending rename: before
+  abstract public void setUp();
+
+  // HIVE-14444 pending rename: after
+  abstract public void tearDown();
+
+  // HIVE-14444 pending rename: afterClass
+  abstract public void shutdown() throws Exception;
+
+  abstract public void runTest(String name, String name2, String absolutePath) throws Exception;
+
+  public final TestRule buildClassRule() {
+    return new TestRule() {
+      @Override
+      public Statement apply(final Statement base, Description description) {
+        return new Statement() {
+          @Override
+          public void evaluate() throws Throwable {
+            CliAdapter.this.beforeClass();
+            try {
+              base.evaluate();
+            } finally {
+              CliAdapter.this.shutdown();
+            }
+          }
+        };
+      }
+    };
+  }
+
+  public final TestRule buildTestRule() {
+    return new TestRule() {
+      @Override
+      public Statement apply(final Statement base, Description description) {
+        return new Statement() {
+          @Override
+          public void evaluate() throws Throwable {
+            CliAdapter.this.setUp();
+            try {
+              base.evaluate();
+            } finally {
+              CliAdapter.this.tearDown();
+            }
+          }
+        };
+      }
+    };
+  }
+
+  // HIVE-14444: pending refactor to push File forward
+  public final void runTest(String name, File qfile) throws Exception {
+    runTest(name, qfile.getName(), qfile.getAbsolutePath());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
new file mode 100644
index 0000000..e7af372
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
@@ -0,0 +1,503 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli.control;
+
+import java.io.File;
+import java.net.MalformedURLException;
+import java.net.URL;
+
+import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
+import org.apache.hadoop.hive.ql.parse.CoreParseNegative;
+
+public class CliConfigs {
+
+  private static URL testConfigProps = getTestPropsURL();
+
+  private static URL getTestPropsURL() {
+    try {
+      return new File(
+          AbstractCliConfig.HIVE_ROOT + "/itests/src/test/resources/testconfiguration.properties")
+              .toURI().toURL();
+    } catch (MalformedURLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public static class CliConfig extends AbstractCliConfig {
+
+    public CliConfig() {
+      super(CoreCliDriver.class);
+      try {
+        setQueryDir("ql/src/test/queries/clientpositive");
+
+        excludesFrom(testConfigProps, "minillap.query.files");
+        excludesFrom(testConfigProps, "minimr.query.files");
+        excludesFrom(testConfigProps, "minitez.query.files");
+        excludesFrom(testConfigProps, "encrypted.query.files");
+        excludesFrom(testConfigProps, "spark.only.query.files");
+        excludesFrom(testConfigProps, "disabled.query.files");
+
+        setResultsDir("ql/src/test/results/clientpositive");
+        setLogDir("itests/qtest/target/qfile-results/clientpositive");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("");
+        setClusterType(MiniClusterType.none);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class ParseNegativeConfig extends AbstractCliConfig {
+    public ParseNegativeConfig() {
+      super(CoreParseNegative.class);
+      try {
+        setQueryDir("ql/src/test/queries/negative");
+
+        setResultsDir("ql/src/test/results/compiler/errors");
+        setLogDir("itests/qtest/target/qfile-results/negative");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("data/conf/perf-reg/");
+        setClusterType(MiniClusterType.none);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class MinimrCliConfig extends AbstractCliConfig {
+    public MinimrCliConfig() {
+      super(CoreCliDriver.class);
+      try {
+        setQueryDir("ql/src/test/queries/clientpositive");
+
+        includesFrom(testConfigProps, "minimr.query.files");
+
+        setResultsDir("ql/src/test/results/clientpositive");
+        setLogDir("itests/qtest/target/qfile-results/clientpositive");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("");
+        setClusterType(MiniClusterType.mr);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class MiniTezCliConfig extends AbstractCliConfig {
+    public MiniTezCliConfig() {
+      super(CoreCliDriver.class);
+      try {
+        setQueryDir("ql/src/test/queries/clientpositive");
+
+        includesFrom(testConfigProps, "minitez.query.files");
+        includesFrom(testConfigProps, "minitez.query.files.shared");
+
+        setResultsDir("ql/src/test/results/clientpositive/tez");
+        setLogDir("itests/qtest/target/qfile-results/clientpositive");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("data/conf/tez");
+        setClusterType(MiniClusterType.tez);
+        setMetastoreType(MetastoreType.hbase);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class MiniLlapCliConfig extends AbstractCliConfig {
+    public MiniLlapCliConfig() {
+      super(CoreCliDriver.class);
+      try {
+        setQueryDir("ql/src/test/queries/clientpositive");
+
+        includesFrom(testConfigProps, "minillap.query.files");
+        includesFrom(testConfigProps, "minillap.shared.query.files");
+
+        setResultsDir("ql/src/test/results/clientpositive/llap");
+        setLogDir("itests/qtest/target/qfile-results/clientpositive");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("data/conf/llap");
+        setClusterType(MiniClusterType.llap);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class EncryptedHDFSCliConfig extends AbstractCliConfig {
+    public EncryptedHDFSCliConfig() {
+      super(CoreCliDriver.class);
+      try {
+        setQueryDir("ql/src/test/queries/clientpositive");
+
+        includesFrom(testConfigProps, "encrypted.query.files");
+
+        setResultsDir("ql/src/test/results/clientpositive/encrypted");
+        setLogDir("itests/qtest/target/qfile-results/clientpositive");
+
+        setInitScript("q_test_init_for_encryption.sql");
+        setCleanupScript("q_test_cleanup_for_encryption.sql");
+
+        setHiveConfDir("data/conf");
+        setClusterType(MiniClusterType.encrypted);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class ContribCliConfig extends AbstractCliConfig {
+    public ContribCliConfig() {
+      super(CoreCliDriver.class);
+      try {
+        setQueryDir("contrib/src/test/queries/clientpositive");
+
+        setResultsDir("contrib/src/test/results/clientpositive");
+        setLogDir("itests/qtest/target/qfile-results/contribclientpositive");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("");
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class PerfCliConfig extends AbstractCliConfig {
+    public PerfCliConfig() {
+      super(CorePerfCliDriver.class);
+      try {
+        setQueryDir("ql/src/test/queries/clientpositive/perf");
+
+        excludesFrom(testConfigProps, "minimr.query.files");
+        excludesFrom(testConfigProps, "minitez.query.files");
+        excludesFrom(testConfigProps, "encrypted.query.files");
+
+        setResultsDir("ql/src/test/results/clientpositive/perf/");
+        setLogDir("itests/qtest/target/qfile-results/clientpositive/");
+
+        setInitScript("q_perf_test_init.sql");
+        setCleanupScript("q_perf_test_cleanup.sql");
+
+        setHiveConfDir("data/conf/perf-reg/");
+        setClusterType(MiniClusterType.tez);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class CompareCliConfig extends AbstractCliConfig {
+    public CompareCliConfig() {
+      super(CoreCompareCliDriver.class);
+      try {
+        setQueryDir("ql/src/test/queries/clientcompare");
+
+        setResultsDir("ql/src/test/results/clientcompare");
+        setLogDir("itests/qtest/target/qfile-results/clientcompare");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("");
+        setClusterType(MiniClusterType.none);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class NegativeCliConfig extends AbstractCliConfig {
+    public NegativeCliConfig() {
+      super(CoreNegativeCliDriver.class);
+      try {
+        setQueryDir("ql/src/test/queries/clientnegative");
+
+        excludesFrom(testConfigProps, "minimr.query.negative.files");
+        excludeQuery("authorization_uri_import.q");
+
+        setResultsDir("ql/src/test/results/clientnegative");
+        setLogDir("itests/qtest/target/qfile-results/clientnegative");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("");
+        setClusterType(MiniClusterType.none);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class NegativeMinimrCli extends AbstractCliConfig {
+    public NegativeMinimrCli() {
+      super(CoreNegativeCliDriver.class);
+      try {
+        setQueryDir("ql/src/test/queries/clientnegative");
+
+        includesFrom(testConfigProps, "minimr.query.negative.files");
+
+        setResultsDir("ql/src/test/results/clientnegative");
+        setLogDir("itests/qtest/target/qfile-results/clientnegative");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("");
+        setClusterType(MiniClusterType.mr);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class HBaseCliConfig extends AbstractCliConfig {
+    public HBaseCliConfig() {
+      super(CoreHBaseCliDriver.class);
+      try {
+        setQueryDir("hbase-handler/src/test/queries/positive");
+
+        setResultsDir("hbase-handler/src/test/results/positive");
+        setLogDir("itests/qtest/target/qfile-results/hbase-handler/positive");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("");
+        setClusterType(MiniClusterType.none);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class HBaseMinimrCliConfig extends AbstractCliConfig {
+    public HBaseMinimrCliConfig() {
+      super(CoreHBaseCliDriver.class);
+      try {
+        setQueryDir("hbase-handler/src/test/queries/positive");
+        // XXX: i think this was non intentionally set to run only hbase_bulk.m???
+        // includeQuery("hbase_bulk.m"); => will be filter out because not ends with .q
+        // to keep existing behaviour i added this method
+        overrideUserQueryFile("hbase_bulk.m");
+
+        setResultsDir("hbase-handler/src/test/results/positive");
+        setLogDir("itests/qtest/target/qfile-results/hbase-handler/minimrpositive");
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+        setHiveConfDir("");
+        setClusterType(MiniClusterType.mr);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class DummyConfig extends AbstractCliConfig {
+    public DummyConfig() {
+      super(CoreDummy.class);
+      try {
+        setQueryDir("ql/src/test/queries/clientcompare");
+
+        setResultsDir("ql/src/test/results/clientcompare");
+        setLogDir("itests/qtest/target/qfile-results/clientcompare");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("");
+        setClusterType(MiniClusterType.none);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class HBaseNegativeCliConfig extends AbstractCliConfig {
+    public HBaseNegativeCliConfig() {
+      super(CoreHBaseNegativeCliDriver.class);
+      try {
+        setQueryDir("hbase-handler/src/test/queries/negative");
+
+        setResultsDir("hbase-handler/src/test/results/negative");
+        setLogDir("itests/qtest/target/qfile-results/hbase-handler/negative");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("");
+        setClusterType(MiniClusterType.none);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class ContribNegativeCliConfig extends AbstractCliConfig {
+    public ContribNegativeCliConfig() {
+      super(CoreNegativeCliDriver.class);
+      try {
+        setQueryDir("contrib/src/test/queries/clientnegative");
+
+        setResultsDir("contrib/src/test/results/clientnegative");
+        setLogDir("itests/qtest/target/qfile-results/contribclientnegative");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("");
+        setClusterType(MiniClusterType.none);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class BeeLineConfig extends AbstractCliConfig {
+    public BeeLineConfig() {
+      // FIXME: beeline is disabled...
+      super(null);
+      // super(CoreBeeLineDriver.class);
+      try {
+        setQueryDir("ql/src/test/queries/clientpositive");
+
+        excludesFrom(testConfigProps, "beeline.positive.exclude");
+
+        setResultsDir("ql/src/test/results/clientpositive");
+        setLogDir("itests/qtest/target/qfile-results/beelinepositive");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("");
+        setClusterType(MiniClusterType.none);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  // XXX: pending merge of ACC ; and upgrade of executor
+  public static class AccumuloCliConfig extends AbstractCliConfig {
+    public AccumuloCliConfig() {
+      super(null);
+      // super(CoreAccumuloCliDriver.class);
+      try {
+        setQueryDir("accumulo-handler/src/test/queries/positive");
+
+        excludesFrom(testConfigProps, "beeline.positive.exclude");
+
+        setResultsDir("accumulo-handler/src/test/results/positive");
+        setLogDir("itests/qtest/target/qfile-results/accumulo-handler/positive");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("");
+        setClusterType(MiniClusterType.none);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class SparkCliConfig extends AbstractCliConfig {
+    public SparkCliConfig() {
+      super(CoreCliDriver.class);
+      try {
+        setQueryDir("ql/src/test/queries/clientpositive");
+
+        includesFrom(testConfigProps, "spark.query.files");
+
+        setResultsDir("ql/src/test/results/clientpositive/spark");
+        setLogDir("itests/qtest-spark/target/qfile-results/clientpositive/spark");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("data/conf/spark/standalone");
+        setClusterType(MiniClusterType.spark);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class SparkOnYarnCliConfig extends AbstractCliConfig {
+    public SparkOnYarnCliConfig() {
+      super(CoreCliDriver.class);
+      try {
+        setQueryDir("ql/src/test/queries/clientpositive");
+
+        includesFrom(testConfigProps, "miniSparkOnYarn.query.files");
+
+        setResultsDir("ql/src/test/results/clientpositive/spark");
+        setLogDir("itests/qtest-spark/target/qfile-results/clientpositive/spark");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("data/conf/spark/yarn-client");
+        setClusterType(MiniClusterType.miniSparkOnYarn);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+
+  public static class SparkNegativeCliConfig extends AbstractCliConfig {
+    public SparkNegativeCliConfig() {
+      super(CoreNegativeCliDriver.class);
+      try {
+        setQueryDir("ql/src/test/queries/clientnegative");
+
+        includesFrom(testConfigProps, "spark.query.negative.files");
+
+        setResultsDir("ql/src/test/results/clientnegative/spark");
+        setLogDir("itests/qtest-spark/target/qfile-results/clientnegative/spark");
+
+        setInitScript("q_test_init.sql");
+        setCleanupScript("q_test_cleanup.sql");
+
+        setHiveConfDir("data/conf/spark/standalone");
+        setClusterType(MiniClusterType.spark);
+      } catch (Exception e) {
+        throw new RuntimeException("can't construct cliconfig", e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBeeLineDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBeeLineDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBeeLineDriver.java
new file mode 100644
index 0000000..e5144e3
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreBeeLineDriver.java
@@ -0,0 +1,171 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli.control;
+//beeline is excluded by default
+//AFAIK contains broken tests
+//and produces compile errors...i'll comment out this whole class for now...
+/*
+
+import static org.junit.Assert.fail;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.*;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QTestUtil;
+import org.apache.hive.beeline.util.QFileClient;
+import org.apache.hive.service.server.HiveServer2;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+// HIVE-14444: i've dropped this: @RunWith(ConcurrentTestRunner.class)
+public class CoreBeeLineDriver extends CliAdapter {
+  private final String hiveRootDirectory = AbstractCliConfig.HIVE_ROOT;
+  private final String queryDirectory;
+  private final String logDirectory;
+  private final String resultsDirectory;
+  private boolean overwrite = false;
+  private static String scratchDirectory;
+  private static QTestUtil.QTestSetup miniZKCluster = null;
+
+  private static HiveServer2 hiveServer2;
+
+  public CoreBeeLineDriver(AbstractCliConfig testCliConfig) {
+    super(testCliConfig);
+    queryDirectory = testCliConfig.getQueryDirectory();
+    logDirectory = testCliConfig.getLogDir();
+    resultsDirectory = testCliConfig.getResultsDir();
+  }
+
+  @Override
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    HiveConf hiveConf = new HiveConf();
+    hiveConf.logVars(System.err);
+    System.err.flush();
+
+    scratchDirectory = hiveConf.getVar(SCRATCHDIR);
+
+    String testOutputOverwrite = System.getProperty("test.output.overwrite");
+    if (testOutputOverwrite != null && "true".equalsIgnoreCase(testOutputOverwrite)) {
+      overwrite = true;
+    }
+
+    miniZKCluster = new QTestUtil.QTestSetup();
+    miniZKCluster.preTest(hiveConf);
+
+    System.setProperty("hive.zookeeper.quorum",
+        hiveConf.get("hive.zookeeper.quorum"));
+    System.setProperty("hive.zookeeper.client.port",
+        hiveConf.get("hive.zookeeper.client.port"));
+
+    String disableserver = System.getProperty("test.service.disable.server");
+    if (null != disableserver && disableserver.equalsIgnoreCase("true")) {
+      System.err.println("test.service.disable.server=true "
+        + "Skipping HiveServer2 initialization!");
+      return;
+    }
+
+    hiveServer2 = new HiveServer2();
+    hiveServer2.init(hiveConf);
+    System.err.println("Starting HiveServer2...");
+    hiveServer2.start();
+    Thread.sleep(5000);
+  }
+
+
+  @Override
+  @AfterClass
+  public void shutdown() {
+    try {
+      if (hiveServer2 != null) {
+        System.err.println("Stopping HiveServer2...");
+        hiveServer2.stop();
+      }
+    } catch (Throwable t) {
+      t.printStackTrace();
+    }
+
+    if (miniZKCluster != null) {
+      try {
+        miniZKCluster.tearDown();
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }
+  }
+
+  public void runTest(String qFileName) throws Exception {
+    QFileClient qClient = new QFileClient(new HiveConf(), hiveRootDirectory,
+        queryDirectory, logDirectory, resultsDirectory)
+    .setQFileName(qFileName)
+    .setUsername("user")
+    .setPassword("password")
+    .setJdbcUrl("jdbc:hive2://localhost:10000")
+    .setJdbcDriver("org.apache.hive.jdbc.HiveDriver")
+    .setTestDataDirectory(hiveRootDirectory + "/data/files")
+    .setTestScriptDirectory(hiveRootDirectory + "/data/scripts");
+
+    long startTime = System.currentTimeMillis();
+    System.err.println(">>> STARTED " + qFileName
+        + " (Thread " + Thread.currentThread().getName() + ")");
+    try {
+      qClient.run();
+    } catch (Exception e) {
+      System.err.println(">>> FAILED " + qFileName + " with exception:");
+      e.printStackTrace();
+      throw e;
+    }
+    long elapsedTime = (System.currentTimeMillis() - startTime)/1000;
+    String time = "(" + elapsedTime + "s)";
+
+    if (qClient.compareResults()) {
+      System.err.println(">>> PASSED " + qFileName + " " + time);
+    } else {
+      if (qClient.hasErrors()) {
+        System.err.println(">>> FAILED " + qFileName + " (ERROR) " + time);
+        fail();
+      }
+      if (overwrite) {
+        System.err.println(">>> PASSED " + qFileName + " (OVERWRITE) " + time);
+        qClient.overwriteResults();
+      } else {
+        System.err.println(">>> FAILED " + qFileName + " (DIFF) " + time);
+        fail();
+      }
+    }
+  }
+
+  @Override
+  public void setUp() {
+    // TODO Auto-generated method stub
+
+  }
+
+  @Override
+  public void tearDown() {
+    // TODO Auto-generated method stub
+
+  }
+
+  @Override
+  public void runTest(String name, String name2, String absolutePath) throws Exception {
+    runTest(name2);
+  }
+
+}
+
+
+*/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/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
new file mode 100644
index 0000000..5435f9f
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCliDriver.java
@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli.control;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.hive.cli.control.AbstractCliConfig.MetastoreType;
+import org.apache.hadoop.hive.ql.QTestUtil;
+import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+public class CoreCliDriver extends CliAdapter {
+
+  private static QTestUtil qt;
+  
+  public CoreCliDriver(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();
+    boolean useHBaseMetastore = cliConfig.getMetastoreType() == MetastoreType.hbase;
+    try {
+      String hadoopVer = cliConfig.getHadoopVersion();
+      qt = new QTestUtil((cliConfig.getResultsDir()), (cliConfig.getLogDir()), miniMR,
+      hiveConfDir, hadoopVer, initScript, cleanupScript, useHBaseMetastore, true);
+
+      // do a one time initialization
+      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);
+    }
+  }
+
+  @Override
+  @Before
+  public void setUp() {
+    try {
+      qt.clearTestSideEffects();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in setup");
+    }
+  }
+
+  @Override
+  @After
+  public void tearDown() {
+    try {
+      qt.clearPostTestEffects();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in tearDown");
+    }
+  }
+
+  @Override
+  @AfterClass
+  public void shutdown() throws Exception {
+    try {
+      qt.shutdown();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in shutdown");
+    }
+  }
+
+  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();
+    try {
+      System.err.println("Begin query: " + fname);
+
+      qt.addFile(fpath);
+
+      if (qt.shouldBeSkipped(fname)) {
+        System.err.println("Test " + fname + " skipped");
+        return;
+      }
+
+      qt.cliInit(fname, false);
+      int ecode = qt.executeClient(fname);
+      if (ecode != 0) {
+        qt.failed(ecode, fname, debugHint);
+      }
+      ecode = qt.checkCliDriverResults(fname);
+      if (ecode != 0) {
+        qt.failedDiff(ecode, fname, debugHint);
+      }
+    }
+    catch (Throwable e) {
+      qt.failed(e, fname, debugHint);
+    }
+
+    long elapsedTime = System.currentTimeMillis() - startTime;
+    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
+    assertTrue("Test passed", true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/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
new file mode 100644
index 0000000..71a02bc
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreCompareCliDriver.java
@@ -0,0 +1,158 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli.control;
+
+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.QTestUtil;
+import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+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(cliConfig.getResultsDir(), cliConfig.getLogDir(), miniMR,
+      hiveConfDir, hadoopVer, initScript, cleanupScript, false, false);
+
+      // do a one time initialization
+      qt.cleanUp();
+      qt.createSources();
+
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in static initialization");
+    }
+  }
+
+  @Override
+  @Before
+  public void setUp() {
+    try {
+      qt.clearTestSideEffects();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in setup");
+    }
+  }
+
+  @Override
+  @After
+  public void tearDown() {
+    try {
+      qt.clearPostTestEffects();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in tearDown");
+    }
+  }
+
+  @Override
+  @AfterClass
+  public void shutdown() throws Exception {
+    try {
+      qt.shutdown();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in shutdown");
+    }
+  }
+
+  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, "
+     + "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 {
+    final String queryDirectory = cliConfig.getQueryDirectory();
+
+    long startTime = System.currentTimeMillis();
+    try {
+      System.err.println("Begin query: " + fname);
+      // TODO: versions could also be picked at build time.
+      List<String> versionFiles = QTestUtil.getVersionFiles(queryDirectory, tname);
+      if (versionFiles.size() < 2) {
+        fail("Cannot run " + tname + " with only " + versionFiles.size() + " versions");
+      }
+
+      qt.addFile(fpath);
+      for (String versionFile : versionFiles) {
+        qt.addFile(new File(queryDirectory, versionFile), true);
+      }
+
+      if (qt.shouldBeSkipped(fname)) {
+        return;
+      }
+
+      int ecode = 0;
+      List<String> outputs = new ArrayList<>(versionFiles.size());
+      for (String versionFile : versionFiles) {
+        // 1 for "_" after tname; 3 for ".qv" at the end. Version is in between.
+        String versionStr = versionFile.substring(tname.length() + 1, versionFile.length() - 3);
+        outputs.add(qt.cliInit(tname + "." + versionStr, false));
+        // TODO: will this work?
+        ecode = qt.executeClient(versionFile, fname);
+        if (ecode != 0) {
+          qt.failed(ecode, fname, debugHint);
+        }
+      }
+
+      ecode = qt.checkCompareCliDriverResults(fname, outputs);
+      if (ecode != 0) {
+        qt.failedDiff(ecode, fname, debugHint);
+      }
+    }
+    catch (Throwable e) {
+      qt.failed(e, fname, debugHint);
+    }
+
+    long elapsedTime = System.currentTimeMillis() - startTime;
+    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/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
new file mode 100644
index 0000000..b7afb48
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreDummy.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli.control;
+
+import static org.junit.Assert.fail;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.QTestUtil;
+
+public class CoreDummy extends CliAdapter {
+
+  public CoreDummy(AbstractCliConfig cliConfig) {
+    super(cliConfig);
+  }
+
+  @Override
+  public void beforeClass() {
+  }
+
+  @Override
+  public void setUp() {
+  }
+
+  @Override
+  public void tearDown() {
+  }
+
+  @Override
+  public void shutdown() throws Exception {
+  }
+
+  @Override
+  public void runTest(String name, String name2, String absolutePath) throws Exception {
+    List<String> versionFiles = QTestUtil.getVersionFiles(cliConfig.getQueryDirectory(), name);
+    if (versionFiles.size() < 2) {
+      fail("Cannot run " + name2 + " with only " + versionFiles.size() + " versions");
+    }
+    fail("x");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/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
new file mode 100644
index 0000000..956a42d
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseCliDriver.java
@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli.control;
+
+import static org.apache.hadoop.hive.cli.control.AbstractCliConfig.HIVE_ROOT;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.hive.hbase.HBaseQTestUtil;
+import org.apache.hadoop.hive.hbase.HBaseTestSetup;
+import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+public class CoreHBaseCliDriver extends CliAdapter {
+
+  private HBaseQTestUtil qt;
+  private HBaseTestSetup setup = new HBaseTestSetup();
+
+  public CoreHBaseCliDriver(AbstractCliConfig testCliConfig) {
+    super(testCliConfig);
+  }
+
+  @Override
+  @BeforeClass
+  public void beforeClass() {
+        MiniClusterType miniMR = cliConfig.getClusterType();
+        String initScript = cliConfig.getInitScript();
+        String cleanupScript =cliConfig.getCleanupScript();
+
+        try {
+          qt = new HBaseQTestUtil(cliConfig.getResultsDir(), cliConfig.getLogDir(), miniMR,
+          setup, initScript, cleanupScript);
+          qt.cleanUp(null);
+          qt.createSources(null);
+
+        } catch (Exception e) {
+          System.err.println("Exception: " + e.getMessage());
+          e.printStackTrace();
+          System.err.flush();
+          fail("Unexpected exception in static initialization: "+e.getMessage());
+        }
+
+  }
+
+  @Override
+  @Before
+  public void setUp() {
+    try {
+      qt.clearTestSideEffects();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in setup");
+    }
+  }
+  @Override
+  @After
+  public void tearDown() {
+    try {
+      qt.clearPostTestEffects();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in tearDown");
+    }
+  }
+
+  @Override
+  @AfterClass
+  public void shutdown() throws Exception {
+    try {
+      // FIXME: there were 2 afterclass methods...i guess this is the right order...maybe not
+      qt.shutdown();
+      setup.tearDown();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in shutdown");
+    }
+  }
+
+  @Override
+  public void runTest(String tname, String fname, String fpath) throws Exception {
+    long startTime = System.currentTimeMillis();
+    try {
+      System.err.println("Begin query: " + fname);
+
+      qt.addFile(fpath);
+
+      if (qt.shouldBeSkipped(fname)) {
+        System.err.println("Test " + fname + " skipped");
+        return;
+      }
+
+      qt.cliInit(fname, false);
+
+      int ecode = qt.executeClient(fname);
+      if (ecode != 0) {
+        qt.failed(ecode, fname, null);
+      }
+
+      ecode = qt.checkCliDriverResults(fname);
+      if (ecode != 0) {
+        qt.failedDiff(ecode, fname, null);
+      }
+
+    } catch (Throwable e) {
+      qt.failed(e, fname, null);
+    }
+
+    long elapsedTime = System.currentTimeMillis() - startTime;
+    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
+    assertTrue("Test passed", true);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/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
new file mode 100644
index 0000000..6225180
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreHBaseNegativeCliDriver.java
@@ -0,0 +1,121 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.cli.control;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.hive.hbase.HBaseQTestUtil;
+import org.apache.hadoop.hive.hbase.HBaseTestSetup;
+import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+
+public class CoreHBaseNegativeCliDriver extends CliAdapter {
+
+  private HBaseQTestUtil qt;
+  private static HBaseTestSetup setup = new HBaseTestSetup();
+
+  public CoreHBaseNegativeCliDriver(AbstractCliConfig testCliConfig) {
+    super(testCliConfig);
+  }
+
+  @Override
+  public void beforeClass() throws Exception {
+  }
+
+  // hmm..this looks a bit wierd...setup boots qtestutil...this part used to be in beforeclass
+  @Override
+  @Before
+  public void setUp() {
+
+    MiniClusterType miniMR = cliConfig.getClusterType();
+    String initScript = cliConfig.getInitScript();
+    String cleanupScript = cliConfig.getCleanupScript();
+
+    try {
+      qt = new HBaseQTestUtil(cliConfig.getResultsDir(), cliConfig.getLogDir(), miniMR,
+      setup, initScript, cleanupScript);
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in setup");
+    }
+  }
+
+  @Override
+  @After
+  public void tearDown() {
+    try {
+      qt.shutdown();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in tearDown");
+    }
+  }
+
+  @Override
+  @AfterClass
+  public void shutdown() throws Exception {
+    // closeHBaseConnections
+    setup.tearDown();
+  }
+
+  @Override
+  public void runTest(String tname, String fname, String fpath) throws Exception {
+    long startTime = System.currentTimeMillis();
+    try {
+      System.err.println("Begin query: " + fname);
+
+      qt.addFile(fpath);
+
+      if (qt.shouldBeSkipped(fname)) {
+        System.err.println("Test " + fname + " skipped");
+        return;
+      }
+
+      qt.cliInit(fname);
+      qt.clearTestSideEffects();
+      int ecode = qt.executeClient(fname);
+      if (ecode == 0) {
+        qt.failed(fname, null);
+      }
+
+      ecode = qt.checkCliDriverResults(fname);
+      if (ecode != 0) {
+        qt.failedDiff(ecode, fname, null);
+      }
+      qt.clearPostTestEffects();
+
+    } catch (Throwable e) {
+      qt.failed(e, fname, null);
+    }
+
+    long elapsedTime = System.currentTimeMillis() - startTime;
+    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
+    assertTrue("Test passed", true);
+  }
+
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/1f7e10c0/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
new file mode 100644
index 0000000..65b2ce7
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CoreNegativeCliDriver.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.cli.control;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.hive.ql.QTestUtil;
+import org.apache.hadoop.hive.ql.QTestUtil.MiniClusterType;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+
+public class CoreNegativeCliDriver extends CliAdapter{
+
+  private QTestUtil qt;
+  public CoreNegativeCliDriver(AbstractCliConfig testCliConfig) {
+    super(testCliConfig);
+  }
+
+  @Override
+  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((cliConfig.getResultsDir()), (cliConfig.getLogDir()), miniMR,
+       hiveConfDir, hadoopVer, initScript, cleanupScript, false, false);
+      // do a one time initialization
+      qt.cleanUp();
+      qt.createSources();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in static initialization");
+    }
+  }
+
+  @Override
+  @Before
+  public void setUp() {
+    try {
+      qt.clearTestSideEffects();
+    } catch (Throwable e) {
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in setup");
+    }
+  }
+
+  @Override
+  @After
+  public void tearDown() {
+    try {
+      qt.clearPostTestEffects();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in tearDown");
+    }
+  }
+
+  @Override
+  @AfterClass
+  public void shutdown() throws Exception {
+    try {
+      qt.shutdown();
+    } catch (Exception e) {
+      System.err.println("Exception: " + e.getMessage());
+      e.printStackTrace();
+      System.err.flush();
+      fail("Unexpected exception in shutdown");
+    }
+  }
+
+  /**
+   * 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, "
+     + "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();
+    try {
+      System.err.println("Begin query: " + fname);
+
+      qt.addFile(fpath);
+
+      if (qt.shouldBeSkipped(fname)) {
+        System.err.println("Test " + fname + " skipped");
+        return;
+      }
+
+      qt.cliInit(fname, false);
+      int ecode = qt.executeClient(fname);
+      if (ecode == 0) {
+        qt.failed(fname, debugHint);
+      }
+
+      ecode = qt.checkCliDriverResults(fname);
+      if (ecode != 0) {
+        qt.failedDiff(ecode, fname, debugHint);
+      }
+    }
+    catch (Throwable e) {
+      qt.failed(e, fname, debugHint);
+    }
+
+    long elapsedTime = System.currentTimeMillis() - startTime;
+    System.err.println("Done query: " + fname + " elapsedTime=" + elapsedTime/1000 + "s");
+    assertTrue("Test passed", true);
+  }
+}