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);
-  }
-}

Reply via email to