You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by br...@apache.org on 2014/11/26 01:49:35 UTC

svn commit: r1641739 - in /hive/trunk/testutils/ptest2/src: main/java/org/apache/hive/ptest/execution/ main/java/org/apache/hive/ptest/execution/context/ main/java/org/apache/hive/ptest/execution/ssh/ test/java/org/apache/hive/ptest/execution/ test/jav...

Author: brock
Date: Wed Nov 26 00:49:35 2014
New Revision: 1641739

URL: http://svn.apache.org/r1641739
Log:
HIVE-8965 - Enhance PTest to kill all processes between tests and to report when a TEST*.xml file is not generated (Brock reviewed by Szehon)

Modified:
    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/JUnitReportParser.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/Phase.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudExecutionContextProvider.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/SSHCommand.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/SSHCommandExecutor.java
    hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPrepPhase.testExecute.approved.txt
    hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/ssh/TestSSHCommandExecutor.java

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=1641739&r1=1641738&r2=1641739&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 Wed Nov 26 00:49:35 2014
@@ -107,6 +107,10 @@ public class ExecutionPhase extends Phas
         JUnitReportParser parser = new JUnitReportParser(logger, batchLogDir);
         executedTests.addAll(parser.getExecutedTests());
         failedTests.addAll(parser.getFailedTests());
+        // if the TEST*.xml was not generated or was corrupt, let someone know
+        if (parser.getNumAttemptedTests() == 0) {
+          failedTests.add(batch.getName() + " - did not produce a TEST-*.xml file");
+        }
       }
     } 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=1641739&r1=1641738&r2=1641739&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 Wed Nov 26 00:49:35 2014
@@ -29,7 +29,6 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.io.FileUtils;
 import org.apache.hive.ptest.execution.conf.Host;
 import org.apache.hive.ptest.execution.conf.TestBatch;
 import org.apache.hive.ptest.execution.ssh.RSyncCommand;
@@ -218,7 +217,7 @@ class HostExecutor {
     script.delete();
     mLogger.info(drone + " executing " + batch + " with " + command);
     RemoteCommandResult sshResult = new SSHCommand(mSSHCommandExecutor, drone.getPrivateKey(), drone.getUser(),
-        drone.getHost(), drone.getInstance(), command).
+        drone.getHost(), drone.getInstance(), command, true).
         call();
     File batchLogDir = null;
     if(sshResult.getExitCode() == Constants.EXIT_CODE_UNKNOWN) {
@@ -346,8 +345,22 @@ class HostExecutor {
    * exits with a status code of 255 until all drones have been utilized, possibly
    * excluding the host from future use.
    */
+  ListenableFuture<SSHResult> execIgnoreAllErrors(final String cmd)
+      throws Exception {
+    return exec(cmd, false);
+  }
+  /**
+   * Execute command on at least one drone. The method will retry when the command
+   * exits with a status code of 255 until all drones have been utilized, possibly
+   * excluding the host from future use.
+   */
   ListenableFuture<SSHResult> exec(final String cmd)
       throws Exception {
+    return exec(cmd, true);
+  }
+
+  private ListenableFuture<SSHResult> exec(final String cmd, final boolean reportErrors)
+      throws Exception {
     return mExecutor.submit(new Callable<SSHResult>() {
       @Override
       public SSHResult call() throws Exception {
@@ -357,8 +370,8 @@ class HostExecutor {
           templateVariables.put("localDir", drone.getLocalDirectory());
           String command = Templates.getTemplateResult(cmd, templateVariables);
           SSHResult result = new SSHCommand(mSSHCommandExecutor, drone.getPrivateKey(), drone.getUser(),
-              drone.getHost(), drone.getInstance(), command).call();
-          if(result.getExitCode() == Constants.EXIT_CODE_UNKNOWN) {
+              drone.getHost(), drone.getInstance(), command, reportErrors).call();
+          if(reportErrors && result.getExitCode() == Constants.EXIT_CODE_UNKNOWN) {
             mDrones.remove(drone); // return value not checked due to concurrent access
             mLogger.error("Aborting drone during exec " + command,
                 new AbortDroneException("Drone " + drone + " exited with "
@@ -388,7 +401,7 @@ class HostExecutor {
           templateVariables.put("localDir", drone.getLocalDirectory());
           String command = Templates.getTemplateResult(cmd, templateVariables);
           SSHResult result = new SSHCommand(mSSHCommandExecutor, drone.getPrivateKey(), drone.getUser(),
-              drone.getHost(), drone.getInstance(), command).call();
+              drone.getHost(), drone.getInstance(), command, true).call();
           if(result.getExitCode() != Constants.EXIT_CODE_SUCCESS) {
             mDrones.remove(drone); // return value not checked due to concurrent access
             mLogger.error("Aborting drone during exec " + command,

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=1641739&r1=1641738&r2=1641739&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 Wed Nov 26 00:49:35 2014
@@ -76,6 +76,9 @@ public class JUnitReportParser {
     }
     return failedTests;
   }
+  public int getNumAttemptedTests() {
+    return getExecutedTests().size() + getFailedTests().size();
+  }
   private void parse() {
     for(File file : getFiles(directory)) {
       FileInputStream stream = null;

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/Phase.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/Phase.java?rev=1641739&r1=1641738&r2=1641739&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/Phase.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/Phase.java Wed Nov 26 00:49:35 2014
@@ -88,6 +88,14 @@ public abstract class Phase {
     }
     return toListOfResults(futures);
   }
+  protected List<SSHResult> execHostsIgnoreErrors(String command)
+      throws Exception {
+    List<ListenableFuture<SSHResult>> futures = Lists.newArrayList();
+    for(HostExecutor hostExecutor : hostExecutors) {
+      futures.add(hostExecutor.execIgnoreAllErrors(command));
+    }
+    return toListOfResults(futures);
+  }
   // clean prep
   protected List<RemoteCommandResult> execInstances(String command)
       throws Exception {

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java?rev=1641739&r1=1641738&r2=1641739&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java Wed Nov 26 00:49:35 2014
@@ -65,6 +65,8 @@ public class PrepPhase extends Phase {
     Templates.writeTemplateResult("source-prep.vm", sourcePrepScript, getTemplateDefaults());
     execLocally("bash " + sourcePrepScript.getPath());
     logger.debug("Deleting " + sourcePrepScript + ": " + sourcePrepScript.delete());
+    execHostsIgnoreErrors("pkill -f java");
+    execHostsIgnoreErrors("pkill -9 -f java");
     elapsedTime = TimeUnit.MINUTES.convert((System.currentTimeMillis() - start),
         TimeUnit.MILLISECONDS);
     logger.info("PERF: source prep took " + elapsedTime + " minutes");

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudExecutionContextProvider.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudExecutionContextProvider.java?rev=1641739&r1=1641738&r2=1641739&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudExecutionContextProvider.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudExecutionContextProvider.java Wed Nov 26 00:49:35 2014
@@ -286,7 +286,7 @@ public class CloudExecutionContextProvid
             @Override
             public void run() {
               String ip = publicIpOrHostname(node);
-              SSHCommand command = new SSHCommand(mSSHCommandExecutor, mPrivateKey, mUser, ip, 0, "pkill -f java");
+              SSHCommand command = new SSHCommand(mSSHCommandExecutor, mPrivateKey, mUser, ip, 0, "pkill -f java", true);
               mSSHCommandExecutor.execute(command);
               if(command.getExitCode() == Constants.EXIT_CODE_UNKNOWN ||
                   command.getException() != null) {

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/SSHCommand.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/SSHCommand.java?rev=1641739&r1=1641738&r2=1641739&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/SSHCommand.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/SSHCommand.java Wed Nov 26 00:49:35 2014
@@ -22,12 +22,14 @@ public class SSHCommand extends Abstract
 
   private final SSHCommandExecutor executor;
   private final String command;
+  private final boolean reportErrors;
 
   public SSHCommand(SSHCommandExecutor executor, String privateKey,
-      String user, String host, int instance, String command) {
+      String user, String host, int instance, String command, boolean reportErrors) {
     super(privateKey, user, host, instance);
     this.executor = executor;
     this.command = command;
+    this.reportErrors = reportErrors;
   }
 
   @Override
@@ -37,6 +39,10 @@ public class SSHCommand extends Abstract
         getExitCode(), getException(), getOutput());
   }
 
+  public boolean isReportErrors() {
+    return reportErrors;
+  }
+
   public String getCommand() {
     return command;
   }

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/SSHCommandExecutor.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/SSHCommandExecutor.java?rev=1641739&r1=1641738&r2=1641739&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/SSHCommandExecutor.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/SSHCommandExecutor.java Wed Nov 26 00:49:35 2014
@@ -63,7 +63,7 @@ public class SSHCommandExecutor {
           command.setExitCode(Constants.EXIT_CODE_UNKNOWN);
           return;
         }
-        if(attempts++ <= 3 && cmd.getExitCode() == Constants.EXIT_CODE_UNKNOWN) {
+        if(command.isReportErrors() && attempts++ <= 3 && cmd.getExitCode() == Constants.EXIT_CODE_UNKNOWN) {
           mLogger.warn("Command exited with " + cmd.getExitCode() + ", will retry: " + command);
           retry = true;
           TimeUnit.SECONDS.sleep(5);

Modified: hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPrepPhase.testExecute.approved.txt
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPrepPhase.testExecute.approved.txt?rev=1641739&r1=1641738&r2=1641739&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPrepPhase.testExecute.approved.txt (original)
+++ hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPrepPhase.testExecute.approved.txt Wed Nov 26 00:49:35 2014
@@ -1,3 +1,5 @@
 bash /tmp/hive-ptest-units/TestPrepPhase/source-prep.sh
 mkdir -p /some/working/dir/scratch
+pkill -9 -f java
+pkill -f java
 rm -rf /some/working/dir/scratch
\ No newline at end of file

Modified: hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/ssh/TestSSHCommandExecutor.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/ssh/TestSSHCommandExecutor.java?rev=1641739&r1=1641738&r2=1641739&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/ssh/TestSSHCommandExecutor.java (original)
+++ hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/ssh/TestSSHCommandExecutor.java Wed Nov 26 00:49:35 2014
@@ -54,7 +54,7 @@ public class TestSSHCommandExecutor {
       "-o StrictHostKeyChecking=no");
     Assert.assertFalse(executor.isShutdown());
     executor.shutdownNow();
-    SSHCommand command = new SSHCommand(executor, "privateKey", "user", "host", 1, "whoami");
+    SSHCommand command = new SSHCommand(executor, "privateKey", "user", "host", 1, "whoami", true);
     executor.execute(command);
     Assert.assertTrue(executor.isShutdown());
     Assert.assertEquals(Constants.EXIT_CODE_UNKNOWN, command.getExitCode());
@@ -77,7 +77,7 @@ public class TestSSHCommandExecutor {
         return Constants.EXIT_CODE_UNKNOWN;
       }
     });
-    SSHCommand command = new SSHCommand(executor, "privateKey", "user", "host", 1, "whoami");
+    SSHCommand command = new SSHCommand(executor, "privateKey", "user", "host", 1, "whoami", true);
     executor.execute(command);
     Assert.assertTrue(executor.isShutdown());
     Assert.assertEquals(Constants.EXIT_CODE_UNKNOWN, command.getExitCode());