You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ec...@apache.org on 2013/07/25 05:29:42 UTC

svn commit: r1506813 - in /hive/trunk/testutils/ptest2: ./ src/main/java/org/apache/hive/ptest/api/client/ src/main/java/org/apache/hive/ptest/api/request/ src/main/java/org/apache/hive/ptest/api/server/ src/main/java/org/apache/hive/ptest/execution/ s...

Author: ecapriolo
Date: Thu Jul 25 03:29:40 2013
New Revision: 1506813

URL: http://svn.apache.org/r1506813
Log:
HIVE-4892 PTest2 cleanup after merge (Brock Noland via egc)

Submitted by: Brock Noland	
Reviewed by: Edward Capriolo	

Added:
    hive/trunk/testutils/ptest2/src/test/resources/SomeTest-failure.xml   (with props)
    hive/trunk/testutils/ptest2/src/test/resources/SomeTest-success.xml   (with props)
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/SomeTest-truncated.xml   (with props)
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/index_auth.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml   (with props)
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/index_auth.q-hive.log
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml   (with props)
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml   (with props)
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-TestCliDriver.txt
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/union_remove_9.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml   (with props)
Modified:
    hive/trunk/testutils/ptest2/README.md
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/client/PTestClient.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/request/TestStartRequest.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/server/TestExecutor.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JIRAService.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JUnitReportParser.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ReportingPhase.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/QFileTestBatch.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/UnitTestBatch.java
    hive/trunk/testutils/ptest2/src/main/resources/batch-exec.vm
    hive/trunk/testutils/ptest2/src/main/resources/source-prep.vm
    hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.java
    hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.java
    hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportParser.java
    hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportingPhase.testExecute.approved.txt
    hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.java
    hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testBatch.approved.txt
    hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepGit.approved.txt
    hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepNone.approved.txt
    hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepSvn.approved.txt
    hive/trunk/testutils/ptest2/src/test/resources/TEST-SomeTest-failure.xml
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/TEST-SomeTest-truncated.xml
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/TEST-index_auth.q-bucketcontex-ba31fb54-1d7f-4c70-a89d-477b7d155191-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/TEST-skewjoin.q-ab8536a7-1b5c-45ed-ba29-14450f27db8b-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/TEST-skewjoin_union_remove_1.q-6fa31776-d2b0-4e13-9761-11f750627ad1-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/TEST-union_remove_9.q-acb9de8f-1b9c-4874-924c-b2107ca7b07c-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/index_auth.q-bucketcontex-ba31fb54-1d7f-4c70-a89d-477b7d155191-hive.log
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin.q-ab8536a7-1b5c-45ed-ba29-14450f27db8b-TestCliDriver.txt
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin.q-ab8536a7-1b5c-45ed-ba29-14450f27db8b-hive.log
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-6fa31776-d2b0-4e13-9761-11f750627ad1-TestCliDriver.txt
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-6fa31776-d2b0-4e13-9761-11f750627ad1-hive.log
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/union_remove_9.q-acb9de8f-1b9c-4874-924c-b2107ca7b07c-TestCliDriver.txt
    hive/trunk/testutils/ptest2/src/test/resources/test-outputs/union_remove_9.q-acb9de8f-1b9c-4874-924c-b2107ca7b07c-hive.log

Modified: hive/trunk/testutils/ptest2/README.md
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/README.md?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/README.md (original)
+++ hive/trunk/testutils/ptest2/README.md Thu Jul 25 03:29:40 2013
@@ -25,10 +25,10 @@ On all slaves add the following to /etc/
     MaxSessions 100
     MaxStartups 100
 
-# Install git, java, ant, and maven
+# Install git, svn, make, patch, java, ant, and maven
 
-Recent version os java, ant and maven should be installed. Additionally environment variables
-such as MAVEN_OPTS and ANT_OPTS should be configured with large leap sizes:
+Recent version of git, svn, make, patch, java, ant and maven should be installed. Additionally
+environment variables such as MAVEN_OPTS and ANT_OPTS should be configured with large leap sizes:
 
     $ for item in java maven ant; do echo $item; cat /etc/profile.d/${item}.sh;done
     java
@@ -73,7 +73,8 @@ Ensure the user running the tests has st
 
    $ cat ~/.ssh/config
    StrictHostKeyChecking no
-
+   ConnectTimeout 20
+   ServerAliveInterval 1
 
 # Configure properties file
 
@@ -82,4 +83,4 @@ See conf/example-apache-trunk.properties
 # Execute
 
     mvn dependency:copy-dependencies
-    java -Xms4g -Xmx4g -cp "conf/:/home/hiveptest/hive-ptest/target/hive-test-1.0.jar:target/dependency/*" org.apache.hive.ptest.RunTests --properties apache-trunk.properties
+    java -Xms4g -Xmx4g -cp "target/hive-ptest-1.0-classes.jar:target/dependency/*" org.apache.hive.ptest.execution.PTest --properties apache-trunk.properties

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/client/PTestClient.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/client/PTestClient.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/client/PTestClient.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/client/PTestClient.java Thu Jul 25 03:29:40 2013
@@ -80,6 +80,7 @@ public class PTestClient {
   private static final String JIRA = "jira";
   private static final String OUTPUT_DIR = "outputDir";
   private static final String TEST_HANDLE = "testHandle";
+  private static final String CLEAR_LIBRARY_CACHE = "clearLibraryCache";
   private final String mApiEndPoint;
   private final String mLogsEndpoint;
   private final ObjectMapper mMapper;
@@ -102,7 +103,7 @@ public class PTestClient {
         new UsernamePasswordCredentials("hive", password));
   }
   public boolean testStart(String profile, String testHandle,
-      String jira, String patch, String testOutputDir)
+      String jira, String patch, String testOutputDir, boolean clearLibraryCache)
   throws Exception {
     patch = Strings.nullToEmpty(patch).trim();
     if(!patch.isEmpty()) {
@@ -111,7 +112,7 @@ public class PTestClient {
         throw new IllegalArgumentException("Patch " + patch + " was zero bytes");
       }
     }
-    TestStartRequest startRequest = new TestStartRequest(profile, testHandle, jira, patch);
+    TestStartRequest startRequest = new TestStartRequest(profile, testHandle, jira, patch, clearLibraryCache);
     post(startRequest);
     boolean result = false;
     try {
@@ -256,7 +257,8 @@ public class PTestClient {
     options.addOption(null, JIRA, true, "JIRA to post the results to e.g.: HIVE-XXXX");
     options.addOption(null, TEST_HANDLE, true, "Server supplied test handle. (Required for testStop and testTailLog)");
     options.addOption(null, OUTPUT_DIR, true, "Directory to download and save test-results.tar.gz to. (Optional for testStart)");
-
+    options.addOption(null, CLEAR_LIBRARY_CACHE, false, "Before starting the test, delete the ivy and maven directories (Optional for testStart)");
+    
     CommandLine commandLine = parser.parse(options, args);
 
     if(commandLine.hasOption(HELP_SHORT)) {
@@ -278,7 +280,8 @@ public class PTestClient {
           TEST_HANDLE
         });
       result = client.testStart(commandLine.getOptionValue(PROFILE), commandLine.getOptionValue(TEST_HANDLE),
-          commandLine.getOptionValue(JIRA), commandLine.getOptionValue(PATCH), commandLine.getOptionValue(OUTPUT_DIR));
+          commandLine.getOptionValue(JIRA), commandLine.getOptionValue(PATCH), commandLine.getOptionValue(OUTPUT_DIR),
+          commandLine.hasOption(CLEAR_LIBRARY_CACHE));
     } else if("testTailLog".equalsIgnoreCase(command)) {
       result = client.testTailLog(commandLine.getOptionValue(TEST_HANDLE));
     } else if("testList".equalsIgnoreCase(command)) {

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/request/TestStartRequest.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/request/TestStartRequest.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/request/TestStartRequest.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/request/TestStartRequest.java Thu Jul 25 03:29:40 2013
@@ -23,16 +23,18 @@ public class TestStartRequest {
   private String testHandle;
   private String patchURL;
   private String jiraName;
+  private boolean clearLibraryCache;
 
   public TestStartRequest() {
 
   }
   public TestStartRequest(String profile, String testHandle, 
-      String jiraName, String patchURL) {
+      String jiraName, String patchURL, boolean clearLibraryCache) {
     this.profile = profile;
     this.testHandle = testHandle;
     this.jiraName = jiraName;
     this.patchURL = patchURL;
+    this.clearLibraryCache = clearLibraryCache;
   }
   public String getProfile() {
     return profile;
@@ -45,8 +47,13 @@ public class TestStartRequest {
   }
   public void setPatchURL(String patchURL) {
     this.patchURL = patchURL;
+  }  
+  public boolean isClearLibraryCache() {
+    return clearLibraryCache;
+  }
+  public void setClearLibraryCache(boolean clearLibraryCache) {
+    this.clearLibraryCache = clearLibraryCache;
   }
-  
   public String getJiraName() {
     return jiraName;
   }
@@ -64,6 +71,6 @@ public class TestStartRequest {
   public String toString() {
     return "TestStartRequest [profile=" + profile + ", testHandle="
         + testHandle + ", patchURL=" + patchURL + ", jiraName=" + jiraName
-        + "]";
+        + ", clearLibraryCache=" + clearLibraryCache + "]";
   }
 }

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/server/TestExecutor.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/server/TestExecutor.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/server/TestExecutor.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/server/TestExecutor.java Thu Jul 25 03:29:40 2013
@@ -110,6 +110,7 @@ public void run() {
             TestConfiguration testConfiguration = TestConfiguration.fromFile(profileConfFile, logger);
             testConfiguration.setPatch(startRequest.getPatchURL());
             testConfiguration.setJiraName(startRequest.getJiraName());
+            testConfiguration.setClearLibraryCache(startRequest.isClearLibraryCache());
             PTest ptest = mPTestBuilder.build(testConfiguration, mExecutionContext,
                 test.getStartRequest().getTestHandle(), logDir,
                 new LocalCommandFactory(logger), new SSHCommandExecutor(logger),

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java Thu Jul 25 03:29:40 2013
@@ -20,8 +20,8 @@ package org.apache.hive.ptest.execution;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.BlockingQueue;
@@ -41,31 +41,38 @@ import com.google.common.util.concurrent
 
 public class ExecutionPhase extends Phase {
 
+  private final File succeededLogDir;
   private final File failedLogDir;
   private final BlockingQueue<TestBatch> parallelWorkQueue;
   private final BlockingQueue<TestBatch> isolatedWorkQueue;
+  private final Set<String> executedTests;
   private final Set<String> failedTests;
   private final Supplier<List<TestBatch>> testBatchSupplier;
-  private final List<TestBatch> failedTestResults;
+  private final Set<TestBatch> failedTestResults;
 
   public ExecutionPhase(ImmutableList<HostExecutor> hostExecutors,
       LocalCommandFactory localCommandFactory,
       ImmutableMap<String, String> templateDefaults,
-      File failedLogDir, Supplier<List<TestBatch>> testBatchSupplier,
-      Set<String> failedTests, Logger logger) throws IOException {
+      File succeededLogDir, File failedLogDir, Supplier<List<TestBatch>> testBatchSupplier,
+      Set<String> executedTests, Set<String> failedTests, Logger logger)
+          throws IOException {
     super(hostExecutors, localCommandFactory, templateDefaults, logger);
+    this.succeededLogDir = succeededLogDir;
     this.failedLogDir = failedLogDir;
     this.testBatchSupplier = testBatchSupplier;
+    this.executedTests = executedTests;
     this.failedTests = failedTests;
     this.parallelWorkQueue = new LinkedBlockingQueue<TestBatch>();
     this.isolatedWorkQueue = new LinkedBlockingQueue<TestBatch>();
     this.failedTestResults = Collections.
-        synchronizedList(new ArrayList<TestBatch>());
+        synchronizedSet(new HashSet<TestBatch>());
   }
   @Override
 public void execute() throws Throwable {
     long start = System.currentTimeMillis();
+    List<TestBatch> testBatches = Lists.newArrayList();
     for(TestBatch batch : testBatchSupplier.get()) {
+      testBatches.add(batch);
       if(batch.isParallel()) {
         parallelWorkQueue.add(batch);
       } else {
@@ -74,16 +81,17 @@ public void execute() throws Throwable {
     }
     try {
       do {
-        double numberBadHosts = 0d;
+        float numberBadHosts = 0f;
         for(HostExecutor hostExecutor : hostExecutors) {
           if(hostExecutor.remainingDrones() == 0) {
             numberBadHosts++;
           }
         }
         Preconditions.checkState(hostExecutors.size() > 0, "Host executors cannot be empty");
-        if((numberBadHosts / (double)hostExecutors.size()) > 0.30d) {
-          throw new IllegalStateException("Too many bad hosts: " + (int)numberBadHosts + 
-              " bad hosts out of " + hostExecutors.size() + " is greater than threshold of 30%");
+        float percentBadHosts = numberBadHosts / (float)hostExecutors.size();
+        if(percentBadHosts > 0.50f) {
+          throw new IllegalStateException("Too many bad hosts: " + percentBadHosts + "% (" + (int)numberBadHosts + 
+              " / " + hostExecutors.size() + ") is greater than threshold of 50%");
         }
         List<ListenableFuture<Void>> results = Lists.newArrayList();
         for(HostExecutor hostExecutor : getHostExecutors()) {
@@ -93,14 +101,16 @@ public void execute() throws Throwable {
       } while(!(parallelWorkQueue.isEmpty() && isolatedWorkQueue.isEmpty()));
       Preconditions.checkState(parallelWorkQueue.isEmpty(), "Parallel work queue is not empty. All drones must have aborted.");
       Preconditions.checkState(isolatedWorkQueue.isEmpty(), "Isolated work queue is not empty. All drones must have aborted.");
-      if(!failedTestResults.isEmpty()) {
-        for(TestBatch failure : failedTestResults) {
-          File batchLogDir = new File(failedLogDir, failure.getName());
-          JUnitReportParser parser = new JUnitReportParser(logger, batchLogDir);
-          for(String failedTest : parser.getFailedTests()) {
-            failedTests.add(failedTest);
-          }
-        }
+      for(TestBatch batch : testBatches) {
+       File batchLogDir;
+       if(failedTestResults.contains(batch)) {
+         batchLogDir = new File(failedLogDir, batch.getName());
+       } else {
+         batchLogDir = new File(succeededLogDir, batch.getName());
+       }
+       JUnitReportParser parser = new JUnitReportParser(logger, batchLogDir);
+       executedTests.addAll(parser.getExecutedTests());
+       failedTests.addAll(parser.getFailedTests());
       }
     } finally {
       long elapsed = System.currentTimeMillis() - start;

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java Thu Jul 25 03:29:40 2013
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CopyOnWriteArrayList;
@@ -91,7 +92,7 @@ class HostExecutor {
    * @return failed tests
    */
   ListenableFuture<Void> submitTests(final BlockingQueue<TestBatch> parallelWorkQueue,
-      final BlockingQueue<TestBatch> isolatedWorkQueue, final List<TestBatch> failedTestResults) {
+      final BlockingQueue<TestBatch> isolatedWorkQueue, final Set<TestBatch> failedTestResults) {
     return mExecutor.submit(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
@@ -116,7 +117,7 @@ class HostExecutor {
    * are removed the host will be replaced before the next run.
    */
   private void executeTests(final BlockingQueue<TestBatch> parallelWorkQueue,
-      final BlockingQueue<TestBatch> isolatedWorkQueue, final List<TestBatch> failedTestResults)
+      final BlockingQueue<TestBatch> isolatedWorkQueue, final Set<TestBatch> failedTestResults)
           throws Exception {
     mLogger.info("Starting parallel execution on " + mHost.getName());
     List<ListenableFuture<Void>> droneResults = Lists.newArrayList();
@@ -129,7 +130,7 @@ class HostExecutor {
             do {
               batch = parallelWorkQueue.poll(mNumPollSeconds, TimeUnit.SECONDS);
               if(batch != null) {
-                if(!executeTestBatch(drone, batch)) {
+                if(!executeTestBatch(drone, batch, failedTestResults.size())) {
                   failedTestResults.add(batch);
                 }
               }
@@ -154,7 +155,7 @@ class HostExecutor {
         do {
           batch = isolatedWorkQueue.poll(mNumPollSeconds, TimeUnit.SECONDS);
           if(batch != null) {
-            if(!executeTestBatch(drone, batch)) {
+            if(!executeTestBatch(drone, batch, failedTestResults.size())) {
               failedTestResults.add(batch);
             }
           }
@@ -173,7 +174,7 @@ class HostExecutor {
    * Executes the test batch on the drone in question. If the command
    * exits with a status code of 255 throw an AbortDroneException.
    */
-  private boolean executeTestBatch(Drone drone, TestBatch batch)
+  private boolean executeTestBatch(Drone drone, TestBatch batch, int numOfFailedTests)
       throws IOException, SSHExecutionException, AbortDroneException {
     String scriptName = "hiveptest-" + batch.getName() + ".sh";
     File script = new File(mLocalScratchDirectory, scriptName);
@@ -183,6 +184,7 @@ class HostExecutor {
     templateVariables.put("testArguments", batch.getTestArguments());
     templateVariables.put("localDir", drone.getLocalDirectory());
     templateVariables.put("logDir", drone.getLocalLogDirectory());
+    templateVariables.put("numOfFailedTests", String.valueOf(numOfFailedTests));
     String command = Templates.getTemplateResult("bash $localDir/$instanceName/scratch/" + script.getName(),
         templateVariables);
     Templates.writeTemplateResult("batch-exec.vm", script, templateVariables);
@@ -347,4 +349,4 @@ class HostExecutor {
     }
     return result;
   }
-}
\ No newline at end of file
+}

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JIRAService.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JIRAService.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JIRAService.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JIRAService.java Thu Jul 25 03:29:40 2013
@@ -21,8 +21,11 @@ package org.apache.hive.ptest.execution;
 import java.io.IOException;
 import java.net.URL;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
+import org.apache.hive.ptest.api.server.TestLogger;
+import org.apache.hive.ptest.execution.conf.Context;
 import org.apache.hive.ptest.execution.conf.TestConfiguration;
 import org.apache.http.HttpException;
 import org.apache.http.HttpHost;
@@ -44,11 +47,14 @@ import org.apache.http.impl.client.Defau
 import org.apache.http.protocol.BasicHttpContext;
 import org.apache.http.protocol.ExecutionContext;
 import org.apache.http.protocol.HttpContext;
+import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
 class JIRAService {
   private final Logger mLogger;
@@ -71,7 +77,7 @@ class JIRAService {
     mJenkinsURL = configuration.getJenkinsURL();
   }
 
-  void postComment(boolean error, Set<String> failedTests,
+  void postComment(boolean error, int numExecutesTests, Set<String> failedTests,
       List<String> messages) { 
     DefaultHttpClient httpClient = new DefaultHttpClient();    
     try {
@@ -79,7 +85,11 @@ class JIRAService {
       List<String> comments = Lists.newArrayList();
       comments.add("");
       comments.add("");
-      if (error || !failedTests.isEmpty()) {
+      if(!failedTests.isEmpty()) {
+        comments.add("{color:red}Overall{color}: -1 at least one tests failed");
+      } else if(numExecutesTests == 0) {
+        comments.add("{color:red}Overall{color}: -1 no tests executed");
+      } else if (error) {
         comments.add("{color:red}Overall{color}: -1 build exited with an error");
       } else {
         comments.add("{color:green}Overall{color}: +1 all checks pass");
@@ -90,26 +100,32 @@ class JIRAService {
         comments.add(mPatch);
       }
       comments.add("");
-      if (failedTests.isEmpty()) {
-        comments.add(formatSuccess("+1 all tests passed"));
-      } else {
-        comments.add(formatError("-1 due to " + failedTests.size()
-            + " failed/errored test(s)"));
-        comments.add("Failed tests:");
-        comments.addAll(failedTests);
+      if(numExecutesTests > 0) {
+        if (failedTests.isEmpty()) {
+          comments.add(formatSuccess("+1 "+ numExecutesTests + " tests passed"));
+        } else {
+          comments.add(formatError("-1 due to " + failedTests.size()
+              + " failed/errored test(s), " + numExecutesTests + " tests executed"));
+          comments.add("*Failed tests:*");
+          comments.add("{noformat}");
+          comments.addAll(failedTests);
+          comments.add("{noformat}");
+        }
+        comments.add("");        
       }
-      comments.add("");
       comments.add("Test results: " + mJenkinsURL + "/" + buildTag + "/testReport");
       comments.add("Console output: " + mJenkinsURL + "/" + buildTag + "/console");
       comments.add("");
-      comments.add("Messages:");
-      for (String message : messages) {
-        comments.add(message.replaceAll("\n", "\\n"));
+      if(!messages.isEmpty()) {
+        comments.add("Messages:");
+        comments.add("{noformat}");
+        comments.addAll(messages);
+        comments.add("{noformat}");
+        comments.add("");        
       }
-      comments.add("");
       comments.add("This message is automatically generated.");
-      mLogger.info("Comment: " + Joiner.on("\n").join(comments));
-      String body = Joiner.on("\\n").join(comments);
+      mLogger.info("Comment: " + Joiner.on("\n").join(comments));      
+      String body = Joiner.on("\n").join(comments);
       String url = String.format("%s/rest/api/2/issue/%s/comment", mUrl, mName);
       URL apiURL = new URL(mUrl);
       httpClient.getCredentialsProvider()
@@ -121,8 +137,8 @@ class JIRAService {
       localcontext.setAttribute("preemptive-auth", new BasicScheme());
       httpClient.addRequestInterceptor(new PreemptiveAuth(), 0);      
       HttpPost request = new HttpPost(url);
-      StringEntity params = new StringEntity(String.format(
-          "{\"body\": \"%s\"}", body));
+      ObjectMapper mapper = new ObjectMapper();
+      StringEntity params = new StringEntity(mapper.writeValueAsString(new Body(body)));
       request.addHeader("Content-Type", "application/json");
       request.setEntity(params);
       HttpResponse httpResponse = httpClient.execute(request, localcontext);
@@ -140,6 +156,23 @@ class JIRAService {
     }
   }
   
+  @SuppressWarnings("unused")  
+  private static class Body {
+    private String body;
+    public Body() {
+      
+    }
+    public Body(String body) {
+      this.body = body;
+    }
+    public String getBody() {
+      return body;
+    }
+    public void setBody(String body) {
+      this.body = body;
+    }    
+  }
+  
   /**
    * Hive-Build-123 to Hive-Build/123
    */
@@ -182,4 +215,22 @@ class JIRAService {
       }
     }
   }
-}
\ No newline at end of file
+  
+  public static void main(String[] args) throws Exception {
+    TestLogger logger = new TestLogger(System.err, TestLogger.LEVEL.TRACE);
+    Map<String, String> context = Maps.newHashMap();
+    context.put("jiraUrl", "https://issues.apache.org/jira");
+    context.put("jiraUser", "hiveqa");
+    context.put("jiraPassword", "password goes here");
+    context.put("branch", "trunk");
+    context.put("repository", "repo");
+    context.put("repositoryName", "repoName");
+    context.put("antArgs", "-Dsome=thing");
+    TestConfiguration configuration = new TestConfiguration(new Context(context), logger);
+    configuration.setJiraName("HIVE-4892");
+    JIRAService service = new JIRAService(logger, configuration, "test-123");
+    List<String> messages = Lists.newArrayList("msg1", "msg2");
+    Set<String> failedTests = Sets.newHashSet("failed");
+    service.postComment(false, 5, failedTests, messages);
+  }
+}

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JUnitReportParser.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JUnitReportParser.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JUnitReportParser.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JUnitReportParser.java Thu Jul 25 03:29:40 2013
@@ -36,9 +36,15 @@ import com.google.common.collect.Sets;
 public class JUnitReportParser {
   private final File directory;
   private final Logger logger;
+  private final Set<String> executedTests;
+  private final Set<String> failedTests;
+  private boolean parsed;
   public JUnitReportParser(Logger logger, File directory) throws Exception {
     this.logger = logger;
     this.directory = directory;
+    executedTests = Sets.newHashSet();
+    failedTests =  Sets.newHashSet();
+    parsed = false;
   }
 
   private Set<File> getFiles(File directory) {
@@ -51,15 +57,26 @@ public class JUnitReportParser {
           if(name.startsWith("TEST-") && name.endsWith(".xml")) {
             result.add(file);
           }
-        } else if(file.isDirectory()) {
-          result.addAll(getFiles(file));
         }
       }
     }
     return result;
   }
+  public Set<String> getExecutedTests() {
+    if(!parsed) {
+      parse();
+      parsed = true;
+    }
+    return executedTests;
+  }
   public Set<String> getFailedTests() {
-    final Set<String> failedTests = Sets.newHashSet();
+    if(!parsed) {
+      parse();
+      parsed = true;
+    }
+    return failedTests;
+  }
+  private void parse() {
     for(File file : getFiles(directory)) {
       FileInputStream stream = null;
       try {
@@ -86,8 +103,11 @@ public class JUnitReportParser {
           @Override
         public void endElement(String uri, String localName, String qName)  {
             if ("testcase".equals(qName)) {
-              if(failedOrErrored && name != null) {
-                failedTests.add(name);
+              if(name != null) {
+                executedTests.add(name);
+                if(failedOrErrored) {
+                  failedTests.add(name);
+                }
               }
             }
           }
@@ -104,6 +124,5 @@ public class JUnitReportParser {
         }
       }
     }
-    return failedTests;
   }
-}
\ No newline at end of file
+}

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java Thu Jul 25 03:29:40 2013
@@ -67,6 +67,7 @@ public class PTest {
 
   private final TestConfiguration mConfiguration;
   private final ListeningExecutorService mExecutor;
+  private final Set<String> mExecutedTests;
   private final Set<String> mFailedTests;
   private final List<Phase> mPhases;
   private final ExecutionContext mExecutionContext;
@@ -81,6 +82,7 @@ public class PTest {
     mConfiguration = configuration;
     mLogger = logger;
     mBuildTag = buildTag;
+    mExecutedTests = Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
     mFailedTests = Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
     mExecutionContext = executionContext;
     mExecutor = MoreExecutors.listeningDecorator(Executors.newCachedThreadPool());
@@ -99,6 +101,7 @@ public class PTest {
         put("repositoryName", configuration.getRepositoryName()).
         put("repositoryType", configuration.getRepositoryType()).
         put("branch", configuration.getBranch()).
+        put("clearLibraryCache", String.valueOf(configuration.isClearLibraryCache())).
         put("workingDir", mExecutionContext.getLocalWorkingDirectory()).
         put("antArgs", configuration.getAntArgs()).
         put("buildTag", buildTag).
@@ -120,7 +123,7 @@ public class PTest {
     mPhases.add(new CleanupPhase(mHostExecutors, localCommandFactory, templateDefaults, logger));
     mPhases.add(new PrepPhase(mHostExecutors, localCommandFactory, templateDefaults, scratchDir, patchFile, logger));
     mPhases.add(new ExecutionPhase(mHostExecutors, localCommandFactory, templateDefaults,
-        failedLogDir, testParser.parse(), mFailedTests, logger));
+        succeededLogDir, failedLogDir, testParser.parse(), mExecutedTests, mFailedTests, logger));
     mPhases.add(new ReportingPhase(mHostExecutors, localCommandFactory, templateDefaults, logger));
   }
   public int run() {
@@ -143,11 +146,6 @@ public class PTest {
           elapsedTimes.put(phase.getClass().getSimpleName(), elapsedTime);
         }
       }
-      for(HostExecutor hostExecutor : mHostExecutors) {
-        if(hostExecutor.remainingDrones() == 0) {
-          mExecutionContext.addBadHost(hostExecutor.getHost());
-        }
-      }
       if(!mFailedTests.isEmpty()) {
         throw new TestsFailedException(mFailedTests.size() + " tests failed");
       }
@@ -156,6 +154,11 @@ public class PTest {
       messages.add("Tests failed with: " + throwable.getClass().getSimpleName() + ": " + throwable.getMessage());
       error = true;
     } finally {
+      for(HostExecutor hostExecutor : mHostExecutors) {
+        if(hostExecutor.remainingDrones() == 0) {
+          mExecutionContext.addBadHost(hostExecutor.getHost());
+        }
+      }
       mExecutor.shutdownNow();
       if(mFailedTests.isEmpty()) {
         mLogger.info(String.format("%d failed tests", mFailedTests.size()));
@@ -165,6 +168,7 @@ public class PTest {
       for(String failingTestName : mFailedTests) {
         mLogger.warn(failingTestName);
       }
+      mLogger.info("Executed " + mExecutedTests.size() + " tests");
       for(Map.Entry<String, Long> entry : elapsedTimes.entrySet()) {
         mLogger.info(String.format("PERF: Phase %s took %d minutes", entry.getKey(), entry.getValue()));
       }
@@ -194,7 +198,7 @@ public class PTest {
       return;
     }
     JIRAService jira = new JIRAService(mLogger, mConfiguration, mBuildTag);
-    jira.postComment(error, mFailedTests, messages);
+    jira.postComment(error, mExecutedTests.size(), mFailedTests, messages);
   }
 
   public static class Builder {
@@ -302,4 +306,4 @@ public class PTest {
     }
     System.exit(exitCode);
   }
-}
\ No newline at end of file
+}

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ReportingPhase.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ReportingPhase.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ReportingPhase.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ReportingPhase.java Thu Jul 25 03:29:40 2013
@@ -36,7 +36,7 @@ public class ReportingPhase extends Phas
   @Override
 public void execute() throws Exception {
     execLocally("mkdir $logDir/test-results");
-    execLocally("find $logDir/{failed,succeeded} -name 'TEST*.xml' -exec cp {} $logDir/test-results \\; 2>/dev/null");
+    execLocally("find $logDir/{failed,succeeded} -maxdepth 2 -name 'TEST*.xml' -exec cp {} $logDir/test-results \\; 2>/dev/null");
     execLocally("cd $logDir/ && tar -zvcf test-results.tar.gz test-results/");
   }
-}
\ No newline at end of file
+}

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/QFileTestBatch.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/QFileTestBatch.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/QFileTestBatch.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/QFileTestBatch.java Thu Jul 25 03:29:40 2013
@@ -66,4 +66,49 @@ public class QFileTestBatch implements T
   public boolean isParallel() {
     return isParallel;
   }
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((driver == null) ? 0 : driver.hashCode());
+    result = prime * result + (isParallel ? 1231 : 1237);
+    result = prime * result + ((name == null) ? 0 : name.hashCode());
+    result = prime * result
+        + ((queryFilesProperty == null) ? 0 : queryFilesProperty.hashCode());
+    result = prime * result + ((tests == null) ? 0 : tests.hashCode());
+    return result;
+  }
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    QFileTestBatch other = (QFileTestBatch) obj;
+    if (driver == null) {
+      if (other.driver != null)
+        return false;
+    } else if (!driver.equals(other.driver))
+      return false;
+    if (isParallel != other.isParallel)
+      return false;
+    if (name == null) {
+      if (other.name != null)
+        return false;
+    } else if (!name.equals(other.name))
+      return false;
+    if (queryFilesProperty == null) {
+      if (other.queryFilesProperty != null)
+        return false;
+    } else if (!queryFilesProperty.equals(other.queryFilesProperty))
+      return false;
+    if (tests == null) {
+      if (other.tests != null)
+        return false;
+    } else if (!tests.equals(other.tests))
+      return false;
+    return true;
+  }  
 }

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java Thu Jul 25 03:29:40 2013
@@ -60,6 +60,7 @@ public class TestConfiguration {
   private final String jiraUser;
   private final String jiraPassword;
   private String jiraName;
+  private boolean clearLibraryCache;
 
   @VisibleForTesting
   public TestConfiguration(Context context, Logger logger)
@@ -98,7 +99,12 @@ public class TestConfiguration {
   public void setJiraName(String jiraName) {
     this.jiraName = Strings.nullToEmpty(jiraName);
   }
-
+  public boolean isClearLibraryCache() {
+    return clearLibraryCache; 
+   }
+  public void setClearLibraryCache(boolean clearLibraryCache) {
+    this.clearLibraryCache = clearLibraryCache;
+  }
   public String getJiraUrl() {
     return jiraUrl;
   }

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/UnitTestBatch.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/UnitTestBatch.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/UnitTestBatch.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/UnitTestBatch.java Thu Jul 25 03:29:40 2013
@@ -44,4 +44,30 @@ public class UnitTestBatch implements Te
   public boolean isParallel() {
     return isParallel;
   }
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + (isParallel ? 1231 : 1237);
+    result = prime * result + ((testName == null) ? 0 : testName.hashCode());
+    return result;
+  }
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    UnitTestBatch other = (UnitTestBatch) obj;
+    if (isParallel != other.isParallel)
+      return false;
+    if (testName == null) {
+      if (other.testName != null)
+        return false;
+    } else if (!testName.equals(other.testName))
+      return false;
+    return true;
+  }
 }

Modified: hive/trunk/testutils/ptest2/src/main/resources/batch-exec.vm
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/resources/batch-exec.vm?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/resources/batch-exec.vm (original)
+++ hive/trunk/testutils/ptest2/src/main/resources/batch-exec.vm Thu Jul 25 03:29:40 2013
@@ -40,7 +40,12 @@ cd $localDir/$instanceName/${repositoryN
 ret=$?
 if [[ $ret -ne 0 ]]
 then
-  cp -R $localDir/$instanceName/${repositoryName}-source $logDir/source
+  if [[ $numOfFailedTests -lt 5 ]]
+  then
+    cp -R $localDir/$instanceName/${repositoryName}-source $logDir/source
+  else
+    echo "Number of failed tests $numOfFailedTests exceeded threshold, not copying source"
+  fi
 fi
 if [[ -f $localDir/$instanceName/${repositoryName}-source/build/ql/tmp/hive.log ]]
 then
@@ -57,4 +62,4 @@ then
 else
   echo "$logDir/.log does not exist"
 fi
-exit $ret
\ No newline at end of file
+exit $ret

Modified: hive/trunk/testutils/ptest2/src/main/resources/source-prep.vm
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/resources/source-prep.vm?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/resources/source-prep.vm (original)
+++ hive/trunk/testutils/ptest2/src/main/resources/source-prep.vm Thu Jul 25 03:29:40 2013
@@ -42,7 +42,7 @@ cd $workingDir/
     fi
     cd ${repositoryName}-source
     svn revert -R .
-    rm -rf $(svn status --no-ignore)
+    rm -rf $(svn status --no-ignore | egrep -v '^X|^Performing status on external' | awk '{print $2}')
     svn update
   elif [[ "${repositoryType}" = "git" ]]
   then
@@ -77,7 +77,12 @@ cd $workingDir/
     chmod +x $patchCommandPath
     $patchCommandPath $patchFilePath
   fi
+  if [[ "$clearLibraryCache" == "true" ]]
+  then
+  	rm -rf $workingDir/ivy $workingDir/maven
+  	mkdir $workingDir/ivy $workingDir/maven
+  fi
   ant $antArgs -Divy.default.ivy.user.dir=$workingDir/ivy -Dmvn.local.repo=$workingDir/maven clean package
   ant $antArgs -Divy.default.ivy.user.dir=$workingDir/ivy -Dmvn.local.repo=$workingDir/maven -Dtestcase=nothing test
 ) 2>&1 | tee $logDir/source-prep.txt
-exit ${PIPESTATUS[0]}
\ No newline at end of file
+exit ${PIPESTATUS[0]}

Modified: hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.java (original)
+++ hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.java Thu Jul 25 03:29:40 2013
@@ -48,6 +48,7 @@ public class TestExecutionPhase extends 
   private static final String QFILENAME = "sometest";
   private ExecutionPhase phase;
   private File testDir;
+  private Set<String> executedTests;
   private Set<String> failedTests;
   private List<TestBatch> testBatches;
   private TestBatch testBatch;
@@ -55,13 +56,14 @@ public class TestExecutionPhase extends 
   @Before
   public void setup() throws Exception {
     initialize(getClass().getSimpleName());
+    executedTests = Sets.newHashSet();
     failedTests = Sets.newHashSet();
   }
   private ExecutionPhase getPhase() throws IOException {
     createHostExecutor();
     phase = new ExecutionPhase(hostExecutors, localCommandFactory, templateDefaults,
-        failedLogDir, Suppliers.ofInstance(testBatches),
-        failedTests, logger);
+        succeededLogDir, failedLogDir, Suppliers.ofInstance(testBatches),
+        executedTests, failedTests, logger);
     return phase;
   }
   private void setupQFile(boolean isParallel) throws Exception {
@@ -74,6 +76,12 @@ public class TestExecutionPhase extends 
     testBatch = new UnitTestBatch(DRIVER, false);
     testBatches = Collections.singletonList(testBatch);
   }
+  private void copyTestOutput(String resource, File directory, String name) throws Exception {
+    String junitOutput = Templates.readResource(resource);
+    File junitOutputFile = new File(Dirs.create(
+        new File(directory, name)), "TEST-SomeTest.xml");
+    Files.write(junitOutput.getBytes(Charsets.UTF_8), junitOutputFile);
+  }
   @After
   public void teardown() {
     FileUtils.deleteQuietly(baseDir);
@@ -81,8 +89,10 @@ public class TestExecutionPhase extends 
   @Test
   public void testPassingQFileTest() throws Throwable {
     setupQFile(true);
+    copyTestOutput("SomeTest-success.xml", succeededLogDir, testBatch.getName());
     getPhase().execute();
     Approvals.verify(getExecutedCommands());
+    Assert.assertEquals(Sets.newHashSet("SomeTest." + QFILENAME), executedTests);
     Assert.assertEquals(Sets.newHashSet(), failedTests);
   }
   @Test
@@ -90,19 +100,19 @@ public class TestExecutionPhase extends 
     setupQFile(true);
     sshCommandExecutor.putFailure("bash " + LOCAL_DIR + "/" + HOST + "-" + USER +
         "-0/scratch/hiveptest-" + DRIVER + "-" + QFILENAME + ".sh", 1);
-    String junitOutput = Templates.readResource("TEST-SomeTest-failure.xml");
-    File driverFailureDir = Dirs.create(new File(failedLogDir, testBatch.getName()));
-    File junitOutputFile = new File(driverFailureDir, "TEST-SomeTest-failure.xml");
-    Files.write(junitOutput.getBytes(Charsets.UTF_8), junitOutputFile);
+    copyTestOutput("SomeTest-failure.xml", failedLogDir, testBatch.getName());
     getPhase().execute();
     Approvals.verify(getExecutedCommands());
+    Assert.assertEquals(Sets.newHashSet("SomeTest." + QFILENAME), executedTests);
     Assert.assertEquals(Sets.newHashSet("SomeTest." + QFILENAME), failedTests);
   }
   @Test
   public void testPassingUnitTest() throws Throwable {
     setupUnitTest();
+    copyTestOutput("SomeTest-success.xml", succeededLogDir, testBatch.getName());
     getPhase().execute();
     Approvals.verify(getExecutedCommands());
+    Assert.assertEquals(Sets.newHashSet("SomeTest." + QFILENAME), executedTests);
     Assert.assertEquals(Sets.newHashSet(), failedTests);
   }
   @Test
@@ -110,12 +120,10 @@ public class TestExecutionPhase extends 
     setupUnitTest();
     sshCommandExecutor.putFailure("bash " + LOCAL_DIR + "/" + HOST + "-" + USER +
         "-0/scratch/hiveptest-" + DRIVER + ".sh", 1);
-    String junitOutput = Templates.readResource("TEST-SomeTest-failure.xml");
-    File driverFailureDir = Dirs.create(new File(failedLogDir, testBatch.getName()));
-    File junitOutputFile = new File(driverFailureDir, "TEST-SomeTest-failure.xml");
-    Files.write(junitOutput.getBytes(Charsets.UTF_8), junitOutputFile);
+    copyTestOutput("SomeTest-failure.xml", failedLogDir, testBatch.getName());
     getPhase().execute();
     Approvals.verify(getExecutedCommands());
+    Assert.assertEquals(Sets.newHashSet("SomeTest." + QFILENAME), executedTests);
     Assert.assertEquals(Sets.newHashSet("SomeTest." + QFILENAME), failedTests);
   }
-}
\ No newline at end of file
+}

Modified: hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.java (original)
+++ hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.java Thu Jul 25 03:29:40 2013
@@ -24,6 +24,7 @@ import static org.mockito.Mockito.spy;
 import java.io.File;
 import java.util.Collections;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Executors;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -46,6 +47,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 
@@ -86,7 +88,7 @@ public class TestHostExecutor {
   private Logger logger;
   private BlockingQueue<TestBatch> parallelWorkQueue;
   private BlockingQueue<TestBatch> isolatedWorkQueue;
-  private List<TestBatch> failedTestResults;
+  private Set<TestBatch> failedTestResults;
   private TestBatch testBatchParallel1;
   private TestBatch testBatchParallel2;
   private TestBatch testBatchIsolated1;
@@ -101,7 +103,7 @@ public class TestHostExecutor {
     failedLogDir = Dirs.create(new File(logDir, "failed"));
     parallelWorkQueue = new LinkedBlockingQueue<TestBatch>();
     isolatedWorkQueue = new LinkedBlockingQueue<TestBatch>();
-    failedTestResults = Lists.newArrayList();
+    failedTestResults = Sets.newHashSet();
     testBatchParallel1 = new UnitTestBatch(DRIVER_PARALLEL_1, true);
     testBatchParallel2 = new UnitTestBatch(DRIVER_PARALLEL_2, true);
     testBatchIsolated1 = new UnitTestBatch(DRIVER_ISOLATED_1, false);
@@ -151,7 +153,7 @@ public class TestHostExecutor {
     parallelWorkQueue.addAll(Lists.newArrayList(testBatchParallel1, testBatchParallel2));
     parallelWorkQueue.addAll(Lists.newArrayList(testBatchIsolated1, testBatchIsolated2));
     executor.submitTests(parallelWorkQueue, isolatedWorkQueue, failedTestResults).get();
-    Assert.assertEquals(Collections.emptyList(),  failedTestResults);
+    Assert.assertEquals(Collections.emptySet(),  failedTestResults);
     Approvals.verify(getExecutedCommands());
   }
   @Test
@@ -162,7 +164,7 @@ public class TestHostExecutor {
     HostExecutor executor = createHostExecutor();
     parallelWorkQueue.addAll(Lists.newArrayList(testBatchParallel1));
     executor.submitTests(parallelWorkQueue, isolatedWorkQueue, failedTestResults).get();
-    Assert.assertEquals(Collections.emptyList(),  failedTestResults);
+    Assert.assertEquals(Collections.emptySet(),  failedTestResults);
     Assert.assertTrue(parallelWorkQueue.toString(), parallelWorkQueue.isEmpty());
     Approvals.verify(getExecutedCommands());
   }
@@ -174,7 +176,7 @@ public class TestHostExecutor {
     HostExecutor executor = createHostExecutor();
     isolatedWorkQueue.addAll(Lists.newArrayList(testBatchIsolated1));
     executor.submitTests(parallelWorkQueue, isolatedWorkQueue, failedTestResults).get();
-    Assert.assertEquals(Collections.emptyList(),  failedTestResults);
+    Assert.assertEquals(Collections.emptySet(),  failedTestResults);
     Assert.assertTrue(isolatedWorkQueue.toString(), parallelWorkQueue.isEmpty());
     Approvals.verify(getExecutedCommands());
   }
@@ -186,7 +188,7 @@ public class TestHostExecutor {
     HostExecutor executor = createHostExecutor();
     parallelWorkQueue.addAll(Lists.newArrayList(testBatchParallel1));
     executor.submitTests(parallelWorkQueue, isolatedWorkQueue, failedTestResults).get();
-    Assert.assertEquals(Collections.emptyList(),  failedTestResults);
+    Assert.assertEquals(Collections.emptySet(),  failedTestResults);
     Assert.assertTrue(parallelWorkQueue.toString(), parallelWorkQueue.isEmpty());
     Approvals.verify(getExecutedCommands());
   }
@@ -198,7 +200,7 @@ public class TestHostExecutor {
     HostExecutor executor = createHostExecutor();
     isolatedWorkQueue.addAll(Lists.newArrayList(testBatchIsolated1));
     executor.submitTests(parallelWorkQueue, isolatedWorkQueue, failedTestResults).get();
-    Assert.assertEquals(Collections.emptyList(),  failedTestResults);
+    Assert.assertEquals(Collections.emptySet(),  failedTestResults);
     Assert.assertTrue(isolatedWorkQueue.toString(), parallelWorkQueue.isEmpty());
     Approvals.verify(getExecutedCommands());
   }
@@ -210,7 +212,7 @@ public class TestHostExecutor {
     HostExecutor executor = createHostExecutor();
     isolatedWorkQueue.addAll(Lists.newArrayList(testBatchIsolated1));
     executor.submitTests(parallelWorkQueue, isolatedWorkQueue, failedTestResults).get();
-    Assert.assertEquals(Collections.emptyList(),  failedTestResults);
+    Assert.assertEquals(Collections.emptySet(),  failedTestResults);
     Assert.assertTrue(isolatedWorkQueue.toString(), parallelWorkQueue.isEmpty());
     Approvals.verify(getExecutedCommands());
   }

Modified: hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportParser.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportParser.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportParser.java (original)
+++ hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportParser.java Thu Jul 25 03:29:40 2013
@@ -22,19 +22,41 @@ import java.io.File;
 
 import junit.framework.Assert;
 
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Sets;
+import com.google.common.io.Files;
 
 public class TestReportParser {
   private static final Logger LOG = LoggerFactory
       .getLogger(TestReportParser.class);
+  private File baseDir;
+  @Before
+  public void setup() {
+    baseDir = Files.createTempDir();
+  }
+  @After
+  public void teardown() {
+    if(baseDir != null) {
+      FileUtils.deleteQuietly(baseDir);
+    }
+  }
   @Test
   public void test() throws Exception {
     File reportDir = new File("src/test/resources/test-outputs");
-    JUnitReportParser parser = new JUnitReportParser(LOG, reportDir);
+    for(File file : reportDir.listFiles()) {
+      if(file.getName().endsWith(".xml")) {
+        Files.copy(file, new File(baseDir, "TEST-" + file.getName()));
+      } else {
+        Files.copy(file, new File(baseDir, file.getName()));
+      }
+    }
+    JUnitReportParser parser = new JUnitReportParser(LOG, baseDir);
     Assert.assertEquals(3, parser.getFailedTests().size());
     Assert.assertEquals(Sets.
         newHashSet("org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_skewjoin_union_remove_1",

Modified: hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportingPhase.testExecute.approved.txt
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportingPhase.testExecute.approved.txt?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportingPhase.testExecute.approved.txt (original)
+++ hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportingPhase.testExecute.approved.txt Thu Jul 25 03:29:40 2013
@@ -1,3 +1,3 @@
 cd /tmp/hive-ptest-units/TestReportingPhase/logs/ && tar -zvcf test-results.tar.gz test-results/
-find /tmp/hive-ptest-units/TestReportingPhase/logs/{failed,succeeded} -name 'TEST*.xml' -exec cp {} /tmp/hive-ptest-units/TestReportingPhase/logs/test-results \; 2>/dev/null
-mkdir /tmp/hive-ptest-units/TestReportingPhase/logs/test-results
\ No newline at end of file
+find /tmp/hive-ptest-units/TestReportingPhase/logs/{failed,succeeded} -maxdepth 2 -name 'TEST*.xml' -exec cp {} /tmp/hive-ptest-units/TestReportingPhase/logs/test-results \; 2>/dev/null
+mkdir /tmp/hive-ptest-units/TestReportingPhase/logs/test-results

Modified: hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.java?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.java (original)
+++ hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.java Thu Jul 25 03:29:40 2013
@@ -67,7 +67,9 @@ public class TestScripts  {
     templateVariables.put("logDir", "/some/log/dir");
     templateVariables.put("instanceName", "instance-1");
     templateVariables.put("batchName","batch-1");
+    templateVariables.put("numOfFailedTests", "20");
     templateVariables.put("testArguments", "-Dtest=arg1");
+    templateVariables.put("clearLibraryCache", "true");
     templateVariables.put("javaHome", "/usr/java/jdk1.7");
     templateVariables.put("antEnvOpts", "-Dhttp.proxyHost=somehost -Dhttp.proxyPort=3128");
     String template = readResource("batch-exec.vm");
@@ -86,6 +88,7 @@ public class TestScripts  {
     templateVariables.put("buildTag", "build-1");
     templateVariables.put("logDir", "/some/log/dir");
     templateVariables.put("testArguments", "-Dtest=arg1");
+    templateVariables.put("clearLibraryCache", "false");
     templateVariables.put("javaHome", "/usr/java/jdk1.7");
     templateVariables.put("antEnvOpts", "-Dhttp.proxyHost=somehost -Dhttp.proxyPort=3128");
     String template = readResource("source-prep.vm");
@@ -104,6 +107,7 @@ public class TestScripts  {
     templateVariables.put("buildTag", "build-1");
     templateVariables.put("logDir", "/some/log/dir");
     templateVariables.put("testArguments", "-Dtest=arg1");
+    templateVariables.put("clearLibraryCache", "true");
     templateVariables.put("javaHome", "/usr/java/jdk1.7");
     templateVariables.put("antEnvOpts", "-Dhttp.proxyHost=somehost -Dhttp.proxyPort=3128");
     templateVariables.put("repositoryType", "git");
@@ -123,6 +127,7 @@ public class TestScripts  {
     templateVariables.put("buildTag", "build-1");
     templateVariables.put("logDir", "/some/log/dir");
     templateVariables.put("testArguments", "-Dtest=arg1");
+    templateVariables.put("clearLibraryCache", "true");
     templateVariables.put("javaHome", "/usr/java/jdk1.7");
     templateVariables.put("antEnvOpts", "-Dhttp.proxyHost=somehost -Dhttp.proxyPort=3128");
     templateVariables.put("repositoryType", "svn");
@@ -147,4 +152,4 @@ public class TestScripts  {
     writer.close();
     return writer.toString();
   }
-}
\ No newline at end of file
+}

Modified: hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testBatch.approved.txt
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testBatch.approved.txt?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testBatch.approved.txt (original)
+++ hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testBatch.approved.txt Thu Jul 25 03:29:40 2013
@@ -39,7 +39,12 @@ cd /some/local/dir/instance-1/apache-sou
 ret=$?
 if [[ $ret -ne 0 ]]
 then
-  cp -R /some/local/dir/instance-1/apache-source /some/log/dir/source
+  if [[ 20 -lt 5 ]]
+  then
+    cp -R /some/local/dir/instance-1/apache-source /some/log/dir/source
+  else
+    echo "Number of failed tests 20 exceeded threshold, not copying source"
+  fi
 fi
 if [[ -f /some/local/dir/instance-1/apache-source/build/ql/tmp/hive.log ]]
 then
@@ -56,4 +61,4 @@ then
 else
   echo "/some/log/dir/.log does not exist"
 fi
-exit $ret
\ No newline at end of file
+exit $ret

Modified: hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepGit.approved.txt
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepGit.approved.txt?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepGit.approved.txt (original)
+++ hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepGit.approved.txt Thu Jul 25 03:29:40 2013
@@ -41,7 +41,7 @@ cd /some/working/dir/
     fi
     cd apache-source
     svn revert -R .
-    rm -rf $(svn status --no-ignore)
+    rm -rf $(svn status --no-ignore | egrep -v '^X|^Performing status on external' | awk '{print $2}')
     svn update
   elif [[ "git" = "git" ]]
   then
@@ -76,7 +76,12 @@ cd /some/working/dir/
     chmod +x $patchCommandPath
     $patchCommandPath $patchFilePath
   fi
+  if [[ "true" == "true" ]]
+  then
+  	rm -rf /some/working/dir/ivy /some/working/dir/maven
+  	mkdir /some/working/dir/ivy /some/working/dir/maven
+  fi
   ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy -Dmvn.local.repo=/some/working/dir/maven clean package
   ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy -Dmvn.local.repo=/some/working/dir/maven -Dtestcase=nothing test
 ) 2>&1 | tee /some/log/dir/source-prep.txt
-exit ${PIPESTATUS[0]}
\ No newline at end of file
+exit ${PIPESTATUS[0]}

Modified: hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepNone.approved.txt
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepNone.approved.txt?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepNone.approved.txt (original)
+++ hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepNone.approved.txt Thu Jul 25 03:29:40 2013
@@ -41,7 +41,7 @@ cd /some/working/dir/
     fi
     cd apache-source
     svn revert -R .
-    rm -rf $(svn status --no-ignore)
+    rm -rf $(svn status --no-ignore | egrep -v '^X|^Performing status on external' | awk '{print $2}')
     svn update
   elif [[ "${repositoryType}" = "git" ]]
   then
@@ -76,7 +76,12 @@ cd /some/working/dir/
     chmod +x $patchCommandPath
     $patchCommandPath $patchFilePath
   fi
+  if [[ "false" == "true" ]]
+  then
+  	rm -rf /some/working/dir/ivy /some/working/dir/maven
+  	mkdir /some/working/dir/ivy /some/working/dir/maven
+  fi
   ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy -Dmvn.local.repo=/some/working/dir/maven clean package
   ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy -Dmvn.local.repo=/some/working/dir/maven -Dtestcase=nothing test
 ) 2>&1 | tee /some/log/dir/source-prep.txt
-exit ${PIPESTATUS[0]}
\ No newline at end of file
+exit ${PIPESTATUS[0]}

Modified: hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepSvn.approved.txt
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepSvn.approved.txt?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepSvn.approved.txt (original)
+++ hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepSvn.approved.txt Thu Jul 25 03:29:40 2013
@@ -41,7 +41,7 @@ cd /some/working/dir/
     fi
     cd apache-source
     svn revert -R .
-    rm -rf $(svn status --no-ignore)
+    rm -rf $(svn status --no-ignore | egrep -v '^X|^Performing status on external' | awk '{print $2}')
     svn update
   elif [[ "svn" = "git" ]]
   then
@@ -76,7 +76,12 @@ cd /some/working/dir/
     chmod +x $patchCommandPath
     $patchCommandPath $patchFilePath
   fi
+  if [[ "true" == "true" ]]
+  then
+  	rm -rf /some/working/dir/ivy /some/working/dir/maven
+  	mkdir /some/working/dir/ivy /some/working/dir/maven
+  fi
   ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy -Dmvn.local.repo=/some/working/dir/maven clean package
   ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy -Dmvn.local.repo=/some/working/dir/maven -Dtestcase=nothing test
 ) 2>&1 | tee /some/log/dir/source-prep.txt
-exit ${PIPESTATUS[0]}
\ No newline at end of file
+exit ${PIPESTATUS[0]}

Added: hive/trunk/testutils/ptest2/src/test/resources/SomeTest-failure.xml
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/SomeTest-failure.xml?rev=1506813&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hive/trunk/testutils/ptest2/src/test/resources/SomeTest-failure.xml
------------------------------------------------------------------------------
    svn:mime-type = application/xml

Added: hive/trunk/testutils/ptest2/src/test/resources/SomeTest-success.xml
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/SomeTest-success.xml?rev=1506813&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hive/trunk/testutils/ptest2/src/test/resources/SomeTest-success.xml
------------------------------------------------------------------------------
    svn:mime-type = application/xml

Modified: hive/trunk/testutils/ptest2/src/test/resources/TEST-SomeTest-failure.xml
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/TEST-SomeTest-failure.xml?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
Binary files - no diff available.

Added: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/SomeTest-truncated.xml
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/SomeTest-truncated.xml?rev=1506813&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/SomeTest-truncated.xml
------------------------------------------------------------------------------
    svn:mime-type = application/xml

Modified: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/TEST-SomeTest-truncated.xml
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/TEST-SomeTest-truncated.xml?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
Binary files - no diff available.

Modified: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/TEST-index_auth.q-bucketcontex-ba31fb54-1d7f-4c70-a89d-477b7d155191-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/TEST-index_auth.q-bucketcontex-ba31fb54-1d7f-4c70-a89d-477b7d155191-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
Binary files - no diff available.

Modified: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/TEST-skewjoin.q-ab8536a7-1b5c-45ed-ba29-14450f27db8b-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/TEST-skewjoin.q-ab8536a7-1b5c-45ed-ba29-14450f27db8b-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
Binary files - no diff available.

Modified: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/TEST-skewjoin_union_remove_1.q-6fa31776-d2b0-4e13-9761-11f750627ad1-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/TEST-skewjoin_union_remove_1.q-6fa31776-d2b0-4e13-9761-11f750627ad1-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
Binary files - no diff available.

Modified: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/TEST-union_remove_9.q-acb9de8f-1b9c-4874-924c-b2107ca7b07c-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/TEST-union_remove_9.q-acb9de8f-1b9c-4874-924c-b2107ca7b07c-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
Binary files - no diff available.

Added: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/index_auth.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/index_auth.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml?rev=1506813&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/index_auth.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
------------------------------------------------------------------------------
    svn:mime-type = application/xml

Modified: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/index_auth.q-bucketcontex-ba31fb54-1d7f-4c70-a89d-477b7d155191-hive.log
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/index_auth.q-bucketcontex-ba31fb54-1d7f-4c70-a89d-477b7d155191-hive.log?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/resources/test-outputs/index_auth.q-bucketcontex-ba31fb54-1d7f-4c70-a89d-477b7d155191-hive.log (original)
+++ hive/trunk/testutils/ptest2/src/test/resources/test-outputs/index_auth.q-bucketcontex-ba31fb54-1d7f-4c70-a89d-477b7d155191-hive.log Thu Jul 25 03:29:40 2013
@@ -1,10 +0,0 @@
-2013-01-27 17:22:57,540 INFO  server.NIOServerCnxnFactory (NIOServerCnxnFactory.java:run(224)) - NIOServerCnxn factory exited run method
-2013-01-27 17:22:57,540 INFO  zookeeper.ClientCnxn (ClientCnxn.java:run(1085)) - Unable to read additional data from server sessionid 0x13c7eb59be60001, likely server has closed socket, closing socket connection and attempting reconnect
-2013-01-27 17:22:57,541 INFO  server.ZooKeeperServer (ZooKeeperServer.java:shutdown(419)) - shutting down
-2013-01-27 17:22:57,541 INFO  server.SessionTrackerImpl (SessionTrackerImpl.java:shutdown(225)) - Shutting down
-2013-01-27 17:22:57,541 INFO  server.PrepRequestProcessor (PrepRequestProcessor.java:shutdown(743)) - Shutting down
-2013-01-27 17:22:57,541 INFO  server.SyncRequestProcessor (SyncRequestProcessor.java:shutdown(175)) - Shutting down
-2013-01-27 17:22:57,541 INFO  server.PrepRequestProcessor (PrepRequestProcessor.java:run(143)) - PrepRequestProcessor exited loop!
-2013-01-27 17:22:57,541 INFO  server.SyncRequestProcessor (SyncRequestProcessor.java:run(155)) - SyncRequestProcessor exited!
-2013-01-27 17:22:57,542 INFO  server.FinalRequestProcessor (FinalRequestProcessor.java:shutdown(415)) - shutdown of request processor complete
-2013-01-27 17:22:57,543 INFO  zookeeper.MiniZooKeeperCluster (MiniZooKeeperCluster.java:shutdown(235)) - Shutdown MiniZK cluster with all ZK servers

Added: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/index_auth.q-hive.log
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/index_auth.q-hive.log?rev=1506813&view=auto
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/resources/test-outputs/index_auth.q-hive.log (added)
+++ hive/trunk/testutils/ptest2/src/test/resources/test-outputs/index_auth.q-hive.log Thu Jul 25 03:29:40 2013
@@ -0,0 +1,10 @@
+2013-01-27 17:22:57,540 INFO  server.NIOServerCnxnFactory (NIOServerCnxnFactory.java:run(224)) - NIOServerCnxn factory exited run method
+2013-01-27 17:22:57,540 INFO  zookeeper.ClientCnxn (ClientCnxn.java:run(1085)) - Unable to read additional data from server sessionid 0x13c7eb59be60001, likely server has closed socket, closing socket connection and attempting reconnect
+2013-01-27 17:22:57,541 INFO  server.ZooKeeperServer (ZooKeeperServer.java:shutdown(419)) - shutting down
+2013-01-27 17:22:57,541 INFO  server.SessionTrackerImpl (SessionTrackerImpl.java:shutdown(225)) - Shutting down
+2013-01-27 17:22:57,541 INFO  server.PrepRequestProcessor (PrepRequestProcessor.java:shutdown(743)) - Shutting down
+2013-01-27 17:22:57,541 INFO  server.SyncRequestProcessor (SyncRequestProcessor.java:shutdown(175)) - Shutting down
+2013-01-27 17:22:57,541 INFO  server.PrepRequestProcessor (PrepRequestProcessor.java:run(143)) - PrepRequestProcessor exited loop!
+2013-01-27 17:22:57,541 INFO  server.SyncRequestProcessor (SyncRequestProcessor.java:run(155)) - SyncRequestProcessor exited!
+2013-01-27 17:22:57,542 INFO  server.FinalRequestProcessor (FinalRequestProcessor.java:shutdown(415)) - shutdown of request processor complete
+2013-01-27 17:22:57,543 INFO  zookeeper.MiniZooKeeperCluster (MiniZooKeeperCluster.java:shutdown(235)) - Shutdown MiniZK cluster with all ZK servers

Added: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml?rev=1506813&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
------------------------------------------------------------------------------
    svn:mime-type = application/xml

Modified: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin.q-ab8536a7-1b5c-45ed-ba29-14450f27db8b-TestCliDriver.txt
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin.q-ab8536a7-1b5c-45ed-ba29-14450f27db8b-TestCliDriver.txt?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin.q-ab8536a7-1b5c-45ed-ba29-14450f27db8b-TestCliDriver.txt (original)
+++ hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin.q-ab8536a7-1b5c-45ed-ba29-14450f27db8b-TestCliDriver.txt Thu Jul 25 03:29:40 2013
@@ -1,10 +0,0 @@
-
-BUILD FAILED
-/data/6/hive-local/p0416.mtv.cloudera.com-hiveptest-2/source/build.xml:349: Keepgoing execution: 1 of 13 iterations failed.
-
-Total time: 17 minutes 15 seconds
-'
-	at com.cloudera.hive.ptest.PTest.run(PTest.java:220)
-	at com.cloudera.hive.ptest.PTest.runAllTests(PTest.java:138)
-	at com.cloudera.hive.ptest.PTest.run(PTest.java:72)
-	at com.cloudera.hive.ptest.PTest.main(PTest.java:345)

Modified: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin.q-ab8536a7-1b5c-45ed-ba29-14450f27db8b-hive.log
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin.q-ab8536a7-1b5c-45ed-ba29-14450f27db8b-hive.log?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin.q-ab8536a7-1b5c-45ed-ba29-14450f27db8b-hive.log (original)
+++ hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin.q-ab8536a7-1b5c-45ed-ba29-14450f27db8b-hive.log Thu Jul 25 03:29:40 2013
@@ -1,10 +0,0 @@
-2013-01-27 17:22:05,609 INFO  server.NIOServerCnxnFactory (NIOServerCnxnFactory.java:run(224)) - NIOServerCnxn factory exited run method
-2013-01-27 17:22:05,609 INFO  server.ZooKeeperServer (ZooKeeperServer.java:shutdown(419)) - shutting down
-2013-01-27 17:22:05,609 INFO  zookeeper.ClientCnxn (ClientCnxn.java:run(1085)) - Unable to read additional data from server sessionid 0x13c7eb6a72e0001, likely server has closed socket, closing socket connection and attempting reconnect
-2013-01-27 17:22:05,609 INFO  server.SessionTrackerImpl (SessionTrackerImpl.java:shutdown(225)) - Shutting down
-2013-01-27 17:22:05,609 INFO  server.PrepRequestProcessor (PrepRequestProcessor.java:shutdown(743)) - Shutting down
-2013-01-27 17:22:05,610 INFO  server.SyncRequestProcessor (SyncRequestProcessor.java:shutdown(175)) - Shutting down
-2013-01-27 17:22:05,610 INFO  server.PrepRequestProcessor (PrepRequestProcessor.java:run(143)) - PrepRequestProcessor exited loop!
-2013-01-27 17:22:05,610 INFO  server.SyncRequestProcessor (SyncRequestProcessor.java:run(155)) - SyncRequestProcessor exited!
-2013-01-27 17:22:05,610 INFO  server.FinalRequestProcessor (FinalRequestProcessor.java:shutdown(415)) - shutdown of request processor complete
-2013-01-27 17:22:05,611 INFO  zookeeper.MiniZooKeeperCluster (MiniZooKeeperCluster.java:shutdown(235)) - Shutdown MiniZK cluster with all ZK servers

Modified: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-6fa31776-d2b0-4e13-9761-11f750627ad1-TestCliDriver.txt
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-6fa31776-d2b0-4e13-9761-11f750627ad1-TestCliDriver.txt?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-6fa31776-d2b0-4e13-9761-11f750627ad1-TestCliDriver.txt (original)
+++ hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-6fa31776-d2b0-4e13-9761-11f750627ad1-TestCliDriver.txt Thu Jul 25 03:29:40 2013
@@ -1,10 +0,0 @@
-
-BUILD FAILED
-/data/6/hive-local/p0415.mtv.cloudera.com-hiveptest-15/source/build.xml:349: Keepgoing execution: 1 of 13 iterations failed.
-
-Total time: 12 minutes 39 seconds
-'
-	at com.cloudera.hive.ptest.PTest.run(PTest.java:220)
-	at com.cloudera.hive.ptest.PTest.runAllTests(PTest.java:138)
-	at com.cloudera.hive.ptest.PTest.run(PTest.java:72)
-	at com.cloudera.hive.ptest.PTest.main(PTest.java:345)

Modified: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-6fa31776-d2b0-4e13-9761-11f750627ad1-hive.log
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-6fa31776-d2b0-4e13-9761-11f750627ad1-hive.log?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-6fa31776-d2b0-4e13-9761-11f750627ad1-hive.log (original)
+++ hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-6fa31776-d2b0-4e13-9761-11f750627ad1-hive.log Thu Jul 25 03:29:40 2013
@@ -1,10 +0,0 @@
-2013-01-27 17:03:09,281 INFO  server.NIOServerCnxnFactory (NIOServerCnxnFactory.java:run(224)) - NIOServerCnxn factory exited run method
-2013-01-27 17:03:09,281 INFO  zookeeper.ClientCnxn (ClientCnxn.java:run(1085)) - Unable to read additional data from server sessionid 0x13c7ea913980001, likely server has closed socket, closing socket connection and attempting reconnect
-2013-01-27 17:03:09,281 INFO  server.ZooKeeperServer (ZooKeeperServer.java:shutdown(419)) - shutting down
-2013-01-27 17:03:09,281 INFO  server.SessionTrackerImpl (SessionTrackerImpl.java:shutdown(225)) - Shutting down
-2013-01-27 17:03:09,281 INFO  server.PrepRequestProcessor (PrepRequestProcessor.java:shutdown(743)) - Shutting down
-2013-01-27 17:03:09,282 INFO  server.SyncRequestProcessor (SyncRequestProcessor.java:shutdown(175)) - Shutting down
-2013-01-27 17:03:09,282 INFO  server.PrepRequestProcessor (PrepRequestProcessor.java:run(143)) - PrepRequestProcessor exited loop!
-2013-01-27 17:03:09,282 INFO  server.SyncRequestProcessor (SyncRequestProcessor.java:run(155)) - SyncRequestProcessor exited!
-2013-01-27 17:03:09,282 INFO  server.FinalRequestProcessor (FinalRequestProcessor.java:shutdown(415)) - shutdown of request processor complete
-2013-01-27 17:03:09,283 INFO  zookeeper.MiniZooKeeperCluster (MiniZooKeeperCluster.java:shutdown(235)) - Shutdown MiniZK cluster with all ZK servers

Added: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml?rev=1506813&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
------------------------------------------------------------------------------
    svn:mime-type = application/xml

Added: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-TestCliDriver.txt
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-TestCliDriver.txt?rev=1506813&view=auto
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-TestCliDriver.txt (added)
+++ hive/trunk/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-TestCliDriver.txt Thu Jul 25 03:29:40 2013
@@ -0,0 +1,10 @@
+
+BUILD FAILED
+/data/6/hive-local/p0415.mtv.cloudera.com-hiveptest-15/source/build.xml:349: Keepgoing execution: 1 of 13 iterations failed.
+
+Total time: 12 minutes 39 seconds
+'
+	at com.cloudera.hive.ptest.PTest.run(PTest.java:220)
+	at com.cloudera.hive.ptest.PTest.runAllTests(PTest.java:138)
+	at com.cloudera.hive.ptest.PTest.run(PTest.java:72)
+	at com.cloudera.hive.ptest.PTest.main(PTest.java:345)

Added: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/union_remove_9.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/union_remove_9.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml?rev=1506813&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/union_remove_9.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml
------------------------------------------------------------------------------
    svn:mime-type = application/xml

Modified: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/union_remove_9.q-acb9de8f-1b9c-4874-924c-b2107ca7b07c-TestCliDriver.txt
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/union_remove_9.q-acb9de8f-1b9c-4874-924c-b2107ca7b07c-TestCliDriver.txt?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/resources/test-outputs/union_remove_9.q-acb9de8f-1b9c-4874-924c-b2107ca7b07c-TestCliDriver.txt (original)
+++ hive/trunk/testutils/ptest2/src/test/resources/test-outputs/union_remove_9.q-acb9de8f-1b9c-4874-924c-b2107ca7b07c-TestCliDriver.txt Thu Jul 25 03:29:40 2013
@@ -1,10 +0,0 @@
-
-BUILD FAILED
-/data/6/hive-local/p0416.mtv.cloudera.com-hiveptest-2/source/build.xml:349: Keepgoing execution: 1 of 13 iterations failed.
-
-Total time: 13 minutes 56 seconds
-'
-	at com.cloudera.hive.ptest.PTest.run(PTest.java:220)
-	at com.cloudera.hive.ptest.PTest.runAllTests(PTest.java:138)
-	at com.cloudera.hive.ptest.PTest.run(PTest.java:72)
-	at com.cloudera.hive.ptest.PTest.main(PTest.java:345)

Modified: hive/trunk/testutils/ptest2/src/test/resources/test-outputs/union_remove_9.q-acb9de8f-1b9c-4874-924c-b2107ca7b07c-hive.log
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/resources/test-outputs/union_remove_9.q-acb9de8f-1b9c-4874-924c-b2107ca7b07c-hive.log?rev=1506813&r1=1506812&r2=1506813&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/resources/test-outputs/union_remove_9.q-acb9de8f-1b9c-4874-924c-b2107ca7b07c-hive.log (original)
+++ hive/trunk/testutils/ptest2/src/test/resources/test-outputs/union_remove_9.q-acb9de8f-1b9c-4874-924c-b2107ca7b07c-hive.log Thu Jul 25 03:29:40 2013
@@ -1,10 +0,0 @@
-2013-01-27 17:04:25,273 INFO  server.NIOServerCnxnFactory (NIOServerCnxnFactory.java:run(224)) - NIOServerCnxn factory exited run method
-2013-01-27 17:04:25,273 INFO  zookeeper.ClientCnxn (ClientCnxn.java:run(1085)) - Unable to read additional data from server sessionid 0x13c7eaabc450001, likely server has closed socket, closing socket connection and attempting reconnect
-2013-01-27 17:04:25,273 INFO  server.ZooKeeperServer (ZooKeeperServer.java:shutdown(419)) - shutting down
-2013-01-27 17:04:25,273 INFO  server.SessionTrackerImpl (SessionTrackerImpl.java:shutdown(225)) - Shutting down
-2013-01-27 17:04:25,273 INFO  server.PrepRequestProcessor (PrepRequestProcessor.java:shutdown(743)) - Shutting down
-2013-01-27 17:04:25,274 INFO  server.SyncRequestProcessor (SyncRequestProcessor.java:shutdown(175)) - Shutting down
-2013-01-27 17:04:25,274 INFO  server.PrepRequestProcessor (PrepRequestProcessor.java:run(143)) - PrepRequestProcessor exited loop!
-2013-01-27 17:04:25,274 INFO  server.SyncRequestProcessor (SyncRequestProcessor.java:run(155)) - SyncRequestProcessor exited!
-2013-01-27 17:04:25,274 INFO  server.FinalRequestProcessor (FinalRequestProcessor.java:shutdown(415)) - shutdown of request processor complete
-2013-01-27 17:04:25,275 INFO  zookeeper.MiniZooKeeperCluster (MiniZooKeeperCluster.java:shutdown(235)) - Shutdown MiniZK cluster with all ZK servers