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 2013/11/01 17:34:57 UTC

svn commit: r1537980 - in /hive/trunk/testutils/ptest2/src: main/java/org/apache/hive/ptest/api/client/ main/java/org/apache/hive/ptest/api/server/ main/java/org/apache/hive/ptest/execution/ main/java/org/apache/hive/ptest/execution/ssh/ test/java/org/...

Author: brock
Date: Fri Nov  1 16:34:56 2013
New Revision: 1537980

URL: http://svn.apache.org/r1537980
Log:
HIVE-5695 - PTest2 fix shutdown, duplicate runs, and add client retry

Added:
    hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/ssh/
    hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/ssh/TestRSyncCommandExecutor.java
    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/api/client/PTestClient.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/server/ExecutionController.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/LocalCommand.java
    hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommandFactory.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/ssh/RSyncCommandExecutor.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/TestHostExecutor.java
    hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testAlternativeTestJVM.approved.txt
    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

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=1537980&r1=1537979&r2=1537980&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 Fri Nov  1 16:34:56 2013
@@ -20,6 +20,7 @@ package org.apache.hive.ptest.api.client
 
 import java.io.File;
 import java.io.FileOutputStream;
+import java.io.IOException;
 import java.net.MalformedURLException;
 import java.net.URL;
 import java.util.concurrent.TimeUnit;
@@ -45,10 +46,12 @@ import org.apache.http.HttpResponse;
 import org.apache.http.StatusLine;
 import org.apache.http.auth.AuthScope;
 import org.apache.http.auth.UsernamePasswordCredentials;
+import org.apache.http.client.HttpRequestRetryHandler;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpPost;
 import org.apache.http.entity.StringEntity;
 import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.protocol.HttpContext;
 import org.apache.http.util.EntityUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 
@@ -81,6 +84,7 @@ public class PTestClient {
   private static final String OUTPUT_DIR = "outputDir";
   private static final String TEST_HANDLE = "testHandle";
   private static final String CLEAR_LIBRARY_CACHE = "clearLibraryCache";
+  private static final int MAX_RETRIES = 10;
   private final String mApiEndPoint;
   private final String mLogsEndpoint;
   private final ObjectMapper mMapper;
@@ -113,7 +117,7 @@ public class PTestClient {
       }
     }
     TestStartRequest startRequest = new TestStartRequest(profile, testHandle, jira, patch, clearLibraryCache);
-    post(startRequest);
+    post(startRequest, false);
     boolean result = false;
     try {
       result = testTailLog(testHandle);
@@ -128,7 +132,7 @@ public class PTestClient {
   public boolean testList()
       throws Exception {
     TestListRequest testListRequest = new TestListRequest();
-    TestListResponse testListResponse = post(testListRequest);
+    TestListResponse testListResponse = post(testListRequest, true);
     for(TestStatus testStatus : testListResponse.getEntries()) {
       System.out.println(testStatus);
     }
@@ -144,7 +148,7 @@ public class PTestClient {
     TestStatusResponse statusResponse;
     do {
       TimeUnit.SECONDS.sleep(5);
-      statusResponse = post(statusRequest);
+      statusResponse = post(statusRequest, true);
     } while(Status.isPending(statusResponse.getTestStatus().getStatus()));
     long offset = 0;
     do {
@@ -154,7 +158,7 @@ public class PTestClient {
       } else {
         TimeUnit.SECONDS.sleep(5);
       }
-      statusResponse = post(statusRequest);
+      statusResponse = post(statusRequest, true);
     } while(Status.isInProgress(statusResponse.getTestStatus().getStatus()));
     while(offset < statusResponse.getTestStatus().getLogFileLength()) {
       offset = printLogs(testHandle, offset);
@@ -185,11 +189,11 @@ public class PTestClient {
   private long printLogs(String testHandle, long offset)
       throws Exception {
     TestLogRequest logsRequest = new TestLogRequest(testHandle, offset, 64 * 1024);
-    TestLogResponse logsResponse = post(logsRequest);
+    TestLogResponse logsResponse = post(logsRequest, true);
     System.out.print(logsResponse.getBody());
     return logsResponse.getOffset();
   }
-  private <S extends GenericResponse> S post(Object payload)
+  private <S extends GenericResponse> S post(Object payload, boolean agressiveRetry)
       throws Exception {
     EndPointResponsePair endPointResponse = Preconditions.
         checkNotNull(REQUEST_TO_ENDPOINT.get(payload.getClass()), payload.getClass().getName());
@@ -199,10 +203,13 @@ public class PTestClient {
       StringEntity params = new StringEntity(payloadString);
       request.addHeader("content-type", "application/json");
       request.setEntity(params);
+      if(agressiveRetry) {
+        mHttpClient.setHttpRequestRetryHandler(new PTestHttpRequestRetryHandler());          
+      }
       HttpResponse httpResponse = mHttpClient.execute(request);
       StatusLine statusLine = httpResponse.getStatusLine();
       if(statusLine.getStatusCode() != 200) {
-        throw new RuntimeException(statusLine.getStatusCode() + " " + statusLine.getReasonPhrase());
+        throw new IllegalStateException(statusLine.getStatusCode() + " " + statusLine.getReasonPhrase());
       }
       String response = EntityUtils.toString(httpResponse.getEntity(), "UTF-8");
       @SuppressWarnings("unchecked")
@@ -223,6 +230,24 @@ public class PTestClient {
       request.abort();
     }
   }
+  private static class PTestHttpRequestRetryHandler implements HttpRequestRetryHandler {
+    @Override
+    public boolean retryRequest(IOException exception, int executionCount,
+        HttpContext context) {
+      System.err.println("LOCAL ERROR: " + exception.getMessage());
+      exception.printStackTrace();
+      if(executionCount > MAX_RETRIES) {
+        return false;
+      }
+      try {
+        Thread.sleep(30L * 1000L);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      }
+      return true;
+    }
+    
+  }
   private static class EndPointResponsePair {
     final String endpoint;
     final Class<? extends GenericResponse> responseClass;

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/server/ExecutionController.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/server/ExecutionController.java?rev=1537980&r1=1537979&r2=1537980&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/server/ExecutionController.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/server/ExecutionController.java Fri Nov  1 16:34:56 2013
@@ -150,7 +150,7 @@ public class ExecutionController {
     String testHandle = stopRequest.getTestHandle();
     Test test = mTests.get(testHandle);
     if(result.hasErrors() ||
-        Strings.emptyToNull(stopRequest.getTestHandle()).trim().isEmpty() ||
+        Strings.nullToEmpty(stopRequest.getTestHandle()).trim().isEmpty() ||
         test == null) {
       return new TestStopResponse(Status.illegalArgument());
     }
@@ -164,7 +164,7 @@ public class ExecutionController {
     String testHandle = stopRequest.getTestHandle();
     Test test = mTests.get(testHandle);
     if(result.hasErrors() ||
-        Strings.emptyToNull(stopRequest.getTestHandle()).trim().isEmpty() ||
+        Strings.nullToEmpty(stopRequest.getTestHandle()).trim().isEmpty() ||
         test == null) {
       return new TestStatusResponse(Status.illegalArgument());
     }
@@ -177,7 +177,7 @@ public class ExecutionController {
     String testHandle = logsRequest.getTestHandle();
     Test testExecution = mTests.get(testHandle);
     if(result.hasErrors() ||
-        Strings.emptyToNull(logsRequest.getTestHandle()).trim().isEmpty() ||
+        Strings.nullToEmpty(logsRequest.getTestHandle()).trim().isEmpty() ||
         testExecution == null ||
         logsRequest.getLength() > MAX_READ_SIZE) {
       return new TestLogResponse(Status.illegalArgument());

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=1537980&r1=1537979&r2=1537980&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 Fri Nov  1 16:34:56 2013
@@ -66,7 +66,8 @@ class HostExecutor {
   private final File mSuccessfulTestLogDir;
   private final File mFailedTestLogDir;
   private final long mNumPollSeconds;
-
+  private volatile boolean mShutdown;
+  
   HostExecutor(Host host, String privateKey, ListeningExecutorService executor,
       SSHCommandExecutor sshCommandExecutor,
       RSyncCommandExecutor rsyncCommandExecutor,
@@ -79,6 +80,7 @@ class HostExecutor {
       drones.add(new Drone(privateKey, host.getUser(), host.getName(),
           index, localDirs[index % localDirs.length]));
     }
+    mShutdown = false;
     mHost = host;
     mDrones = new CopyOnWriteArrayList<Drone>(drones);
     mExecutor = executor;
@@ -116,6 +118,12 @@ class HostExecutor {
   Host getHost() {
     return mHost;
   }
+  void shutdownNow() {
+    this.mShutdown = true;
+  }
+  boolean isShutdown() {
+    return mShutdown;
+  }
   /**
    * Executes parallel test until the parallel work queue is empty. Then
    * executes the isolated tests on the host. During each phase if a
@@ -126,6 +134,10 @@ class HostExecutor {
   private void executeTests(final BlockingQueue<TestBatch> parallelWorkQueue,
       final BlockingQueue<TestBatch> isolatedWorkQueue, final Set<TestBatch> failedTestResults)
           throws Exception {
+    if(mShutdown) {
+      mLogger.warn("Shutting down host " + mHost.getName());
+      return;
+    }
     mLogger.info("Starting parallel execution on " + mHost.getName());
     List<ListenableFuture<Void>> droneResults = Lists.newArrayList();
     for(final Drone drone : ImmutableList.copyOf(mDrones)) {
@@ -136,12 +148,16 @@ class HostExecutor {
           try {
             do {
               batch = parallelWorkQueue.poll(mNumPollSeconds, TimeUnit.SECONDS);
+              if(mShutdown) {
+                mLogger.warn("Shutting down host " + mHost.getName());
+                return null;
+              }
               if(batch != null) {
                 if(!executeTestBatch(drone, batch, failedTestResults)) {
                   failedTestResults.add(batch);
                 }
               }
-            } while(!parallelWorkQueue.isEmpty());
+            } while(!mShutdown && !parallelWorkQueue.isEmpty());
           } catch(AbortDroneException ex) {
             mDrones.remove(drone); // return value not checked due to concurrent access
             mLogger.error("Aborting drone during parallel execution", ex);
@@ -154,6 +170,10 @@ class HostExecutor {
         }
       }));
     }
+    if(mShutdown) {
+      mLogger.warn("Shutting down host " + mHost.getName());
+      return;
+    }
     Futures.allAsList(droneResults).get();
     mLogger.info("Starting isolated execution on " + mHost.getName());
     for(Drone drone : ImmutableList.copyOf(mDrones)) {
@@ -166,7 +186,7 @@ class HostExecutor {
               failedTestResults.add(batch);
             }
           }
-        } while(!isolatedWorkQueue.isEmpty());
+        } while(!mShutdown && !isolatedWorkQueue.isEmpty());
       } catch(AbortDroneException ex) {
         mDrones.remove(drone); // return value not checked due to concurrent access
         mLogger.error("Aborting drone during isolated execution", ex);
@@ -208,6 +228,10 @@ class HostExecutor {
       throw new AbortDroneException("Drone " + drone.toString() + " exited with " +
           Constants.EXIT_CODE_UNKNOWN + ": " + sshResult);
     }
+    if(mShutdown) {
+      mLogger.warn("Shutting down host " + mHost.getName());
+      return false;
+    }
     boolean result;
     if(sshResult.getExitCode() != 0 || sshResult.getException() != null) {
       result = false;

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=1537980&r1=1537979&r2=1537980&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 Fri Nov  1 16:34:56 2013
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.SortedSet;
 
+import org.apache.commons.io.FilenameUtils;
 import org.apache.hive.ptest.api.server.TestLogger;
 import org.apache.hive.ptest.execution.conf.Context;
 import org.apache.hive.ptest.execution.conf.TestConfiguration;
@@ -80,7 +81,7 @@ class JIRAService {
     mJenkinsURL = configuration.getJenkinsURL();
   }
 
-  void postComment(boolean error, int numExecutesTests, SortedSet<String> failedTests,
+  void postComment(boolean error, int numTestsExecuted, SortedSet<String> failedTests,
       List<String> messages) {
     DefaultHttpClient httpClient = new DefaultHttpClient();
     try {
@@ -90,7 +91,7 @@ class JIRAService {
       comments.add("");
       if(!failedTests.isEmpty()) {
         comments.add("{color:red}Overall{color}: -1 at least one tests failed");
-      } else if(numExecutesTests == 0) {
+      } else if(numTestsExecuted == 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");
@@ -103,12 +104,12 @@ class JIRAService {
         comments.add(mPatch);
       }
       comments.add("");
-      if(numExecutesTests > 0) {
+      if(numTestsExecuted > 0) {
         if (failedTests.isEmpty()) {
-          comments.add(formatSuccess("+1 "+ numExecutesTests + " tests passed"));
+          comments.add(formatSuccess("+1 "+ numTestsExecuted + " tests passed"));
         } else {
           comments.add(formatError("-1 due to " + failedTests.size()
-              + " failed/errored test(s), " + numExecutesTests + " tests executed"));
+              + " failed/errored test(s), " + numTestsExecuted + " tests executed"));
           comments.add("*Failed tests:*");
           comments.add("{noformat}");
           comments.addAll(failedTests);
@@ -127,6 +128,11 @@ class JIRAService {
         comments.add("");
       }
       comments.add("This message is automatically generated.");
+      String attachmentId = parseAttachementId(mPatch);
+      if(!attachmentId.isEmpty()) {
+        comments.add("");
+        comments.add("ATTACHMENT ID: " + attachmentId);
+      }
       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);
@@ -225,7 +231,20 @@ class JIRAService {
       }
     }
   }
-
+  private static String parseAttachementId(String patch) {
+    if(patch == null) {
+      return "";
+    }
+    String result = FilenameUtils.getPathNoEndSeparator(patch.trim());
+    if(result == null) {
+      return "";
+    }
+    result = FilenameUtils.getName(result.trim());
+    if(result == null) {
+      return "";
+    }
+    return result.trim();
+  }
   public static void main(String[] args) throws Exception {
     TestLogger logger = new TestLogger(System.err, TestLogger.LEVEL.TRACE);
     Map<String, String> context = Maps.newHashMap();

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java?rev=1537980&r1=1537979&r2=1537980&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommand.java Fri Nov  1 16:34:56 2013
@@ -48,6 +48,12 @@ public class LocalCommand {
       return exitCode;
     }
   }
+  
+  public void kill() {
+    synchronized (process) {
+      process.destroy();
+    }
+  }
 
   public static interface OutputPolicy {
     public void handleOutput(String line);

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommandFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommandFactory.java?rev=1537980&r1=1537979&r2=1537980&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommandFactory.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/LocalCommandFactory.java Fri Nov  1 16:34:56 2013
@@ -24,7 +24,6 @@ import org.slf4j.Logger;
 
 public class LocalCommandFactory {
 
-
   private final Logger mLogger;
 
   public LocalCommandFactory(Logger logger) {

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=1537980&r1=1537979&r2=1537980&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 Fri Nov  1 16:34:56 2013
@@ -77,6 +77,8 @@ public class PTest {
   private final Logger mLogger;
   private final List<HostExecutor> mHostExecutors;
   private final String mBuildTag;
+  private final SSHCommandExecutor mSshCommandExecutor;
+  private final RSyncCommandExecutor mRsyncCommandExecutor;
 
   public PTest(final TestConfiguration configuration, final ExecutionContext executionContext,
       final String buildTag, final File logDir, final LocalCommandFactory localCommandFactory,
@@ -88,6 +90,8 @@ public class PTest {
     mExecutedTests = Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
     mFailedTests = Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
     mExecutionContext = executionContext;
+    mSshCommandExecutor = sshCommandExecutor;
+    mRsyncCommandExecutor = rsyncCommandExecutor;
     mExecutor = MoreExecutors.listeningDecorator(Executors.newCachedThreadPool());
     final File failedLogDir = Dirs.create(new File(logDir, "failed"));
     final File succeededLogDir = Dirs.create(new File(logDir, "succeeded"));
@@ -171,10 +175,13 @@ public class PTest {
       error = true;
     } finally {
       for(HostExecutor hostExecutor : mHostExecutors) {
+        hostExecutor.shutdownNow();
         if(hostExecutor.isBad()) {
           mExecutionContext.addBadHost(hostExecutor.getHost());
         }
       }
+      mSshCommandExecutor.shutdownNow();
+      mRsyncCommandExecutor.shutdownNow();
       mExecutor.shutdownNow();
       SortedSet<String> failedTests = new TreeSet<String>(mFailedTests);
       if(failedTests.isEmpty()) {

Modified: hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/RSyncCommandExecutor.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/RSyncCommandExecutor.java?rev=1537980&r1=1537979&r2=1537980&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/RSyncCommandExecutor.java (original)
+++ hive/trunk/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/RSyncCommandExecutor.java Fri Nov  1 16:34:56 2013
@@ -22,7 +22,9 @@ import java.io.IOException;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.hive.ptest.execution.Constants;
 import org.apache.hive.ptest.execution.LocalCommand;
+import org.apache.hive.ptest.execution.LocalCommandFactory;
 import org.apache.hive.ptest.execution.LocalCommand.CollectPolicy;
 import org.slf4j.Logger;
 
@@ -30,12 +32,20 @@ import org.slf4j.Logger;
 public class RSyncCommandExecutor {
 
   private final Logger mLogger;
+  private final LocalCommandFactory mLocalCommandFactory;
   private final Semaphore mSemaphore;
+  private volatile boolean mShutdown;
 
-  public RSyncCommandExecutor(Logger logger) {
+  public RSyncCommandExecutor(Logger logger, LocalCommandFactory localCommandFactory) {
     mLogger = logger;
-    mSemaphore = new Semaphore(5);
+    mLocalCommandFactory = localCommandFactory;
+    mSemaphore = new Semaphore(Math.min(Runtime.getRuntime().availableProcessors() * 5, 10));
+    mShutdown = false;
+  }
+  public RSyncCommandExecutor(Logger logger) {
+    this(logger, new LocalCommandFactory(logger));
   }
+
   /**
    * Execute the given RSync. If the command exits with a non-zero
    * exit status the command will be retried up to three times.
@@ -52,25 +62,31 @@ public class RSyncCommandExecutor {
       do {
         retry = false;
         if(command.getType() == RSyncCommand.Type.TO_LOCAL) {
-          cmd = new LocalCommand(mLogger, collector,
+          cmd = mLocalCommandFactory.create(collector,
               String.format("timeout 1h rsync -vaPe \"ssh -i %s\" --timeout 600 %s@%s:%s %s",
                   command.getPrivateKey(), command.getUser(), command.getHost(),
                   command.getRemoteFile(), command.getLocalFile()));
         } else if(command.getType() == RSyncCommand.Type.FROM_LOCAL) {
-          cmd = new LocalCommand(mLogger, collector,
+          cmd = mLocalCommandFactory.create(collector,
               String.format("timeout 1h rsync -vaPe \"ssh -i %s\" --timeout 600 --delete --delete-during --force %s %s@%s:%s",
                   command.getPrivateKey(), command.getLocalFile(), command.getUser(), command.getHost(),
                   command.getRemoteFile()));
         } else {
           throw new UnsupportedOperationException(String.valueOf(command.getType()));
         }
+        if(mShutdown) {
+          mLogger.warn("Shutting down command " + command);
+          cmd.kill();
+          command.setExitCode(Constants.EXIT_CODE_UNKNOWN);
+          return;
+        }
         // 12 is timeout and 255 is unspecified error
         if(attempts++ <= 3 && cmd.getExitCode() != 0) {
           mLogger.warn("Command exited with " + cmd.getExitCode() + ", will retry: " + command);
           retry = true;
           TimeUnit.SECONDS.sleep(20);
         }
-      } while (retry); // an error occurred, re-try
+      } while (!mShutdown && retry); // an error occurred, re-try
       command.setExitCode(cmd.getExitCode());
     } catch (IOException e) {
       command.setException(e);
@@ -83,4 +99,10 @@ public class RSyncCommandExecutor {
       command.setOutput(collector.getOutput());
     }
   }
-}
+  boolean isShutdown() {
+    return mShutdown;
+  }
+  public void shutdownNow() {
+    this.mShutdown = true;
+  }
+}
\ No newline at end of file

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=1537980&r1=1537979&r2=1537980&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 Fri Nov  1 16:34:56 2013
@@ -23,14 +23,22 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hive.ptest.execution.Constants;
 import org.apache.hive.ptest.execution.LocalCommand;
 import org.apache.hive.ptest.execution.LocalCommand.CollectPolicy;
+import org.apache.hive.ptest.execution.LocalCommandFactory;
 import org.slf4j.Logger;
 
 public class SSHCommandExecutor {
 
   private final Logger mLogger;
-
-  public SSHCommandExecutor(Logger logger) {
+  private final LocalCommandFactory mLocalCommandFactory;
+  private volatile boolean mShutdown;
+  
+  public SSHCommandExecutor(Logger logger, LocalCommandFactory localCommandFactory) {
     mLogger = logger;
+    mShutdown = false;
+    mLocalCommandFactory = localCommandFactory;
+  }
+  public SSHCommandExecutor(Logger logger) {
+    this(logger, new LocalCommandFactory(logger));
   }
   /**
    * Execute the given command via the ssh command line tool. If the command
@@ -46,13 +54,19 @@ public class SSHCommandExecutor {
       LocalCommand cmd;
       do {
         retry = false;
-        cmd = new LocalCommand(mLogger, collector, commandText);
+        cmd = mLocalCommandFactory.create(collector, commandText);
+        if(mShutdown) {
+          mLogger.warn("Shutting down command " + command);
+          cmd.kill();
+          command.setExitCode(Constants.EXIT_CODE_UNKNOWN);
+          return;
+        }
         if(attempts++ <= 3 && cmd.getExitCode() == Constants.EXIT_CODE_UNKNOWN) {
           mLogger.warn("Command exited with " + cmd.getExitCode() + ", will retry: " + command);
           retry = true;
           TimeUnit.SECONDS.sleep(5);
         }
-      } while (retry); // an error occurred, re-try
+      } while (!mShutdown && retry); // an error occurred, re-try
       command.setExitCode(cmd.getExitCode());
     } catch (Exception e) {
       if(command.getExitCode() == Constants.EXIT_CODE_SUCCESS) {
@@ -63,4 +77,10 @@ public class SSHCommandExecutor {
       command.setOutput(collector.getOutput());
     }
   }
-}
+  boolean isShutdown() {
+    return mShutdown;
+  }
+  public void shutdownNow() {
+    this.mShutdown = true;
+  }
+}
\ 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=1537980&r1=1537979&r2=1537980&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 Fri Nov  1 16:34:56 2013
@@ -193,6 +193,20 @@ public class TestHostExecutor {
     Approvals.verify(getExecutedCommands());
   }
   @Test
+  public void testShutdownBeforeExec()
+      throws Exception {
+    rsyncCommandExecutor.putFailure("/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-parallel-1.sh "
+        + "/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-parallel-1.sh", Constants.EXIT_CODE_UNKNOWN);
+    HostExecutor executor = createHostExecutor();
+    parallelWorkQueue.addAll(Lists.newArrayList(testBatchParallel1));
+    executor.shutdownNow();
+    executor.submitTests(parallelWorkQueue, isolatedWorkQueue, failedTestResults).get();
+    Assert.assertEquals(Collections.emptySet(),  failedTestResults);
+    Assert.assertEquals(parallelWorkQueue.toString(), 1, parallelWorkQueue.size());
+    Approvals.verify(getExecutedCommands());
+    Assert.assertTrue(executor.isShutdown());
+  }
+  @Test
   public void testIsolatedFailsOnRsyncUnknown()
       throws Exception {
     rsyncCommandExecutor.putFailure("/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-isolated-1.sh "+
@@ -201,7 +215,7 @@ public class TestHostExecutor {
     isolatedWorkQueue.addAll(Lists.newArrayList(testBatchIsolated1));
     executor.submitTests(parallelWorkQueue, isolatedWorkQueue, failedTestResults).get();
     Assert.assertEquals(Collections.emptySet(),  failedTestResults);
-    Assert.assertTrue(isolatedWorkQueue.toString(), parallelWorkQueue.isEmpty());
+    Assert.assertTrue(isolatedWorkQueue.toString(), isolatedWorkQueue.isEmpty());
     Approvals.verify(getExecutedCommands());
   }
   @Test

Modified: hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testAlternativeTestJVM.approved.txt
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testAlternativeTestJVM.approved.txt?rev=1537980&r1=1537979&r2=1537980&view=diff
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testAlternativeTestJVM.approved.txt (original)
+++ hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testAlternativeTestJVM.approved.txt Fri Nov  1 16:34:56 2013
@@ -61,7 +61,7 @@ then
   	testModule=./
   fi
   pushd $testModule
-  timeout 2h mvn -o test -Dmaven.repo.local=/some/local/dir/instance-1/maven \
+  timeout 2h mvn -B -o test -Dmaven.repo.local=/some/local/dir/instance-1/maven \
     $mavenArgs $mavenTestArgs -Dtest=arg1 1>/some/log/dir/maven-test.txt 2>&1 </dev/null &
 
   pid=$!

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=1537980&r1=1537979&r2=1537980&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 Fri Nov  1 16:34:56 2013
@@ -61,7 +61,7 @@ then
   	testModule=./
   fi
   pushd $testModule
-  timeout 2h mvn -o test -Dmaven.repo.local=/some/local/dir/instance-1/maven \
+  timeout 2h mvn -B -o test -Dmaven.repo.local=/some/local/dir/instance-1/maven \
     $mavenArgs $mavenTestArgs -Dtest=arg1 1>/some/log/dir/maven-test.txt 2>&1 </dev/null &
 
   pid=$!

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=1537980&r1=1537979&r2=1537980&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 Fri Nov  1 16:34:56 2013
@@ -84,11 +84,11 @@ cd /some/working/dir/
   if [[ "${buildTool}" == "maven" ]]
   then
     rm -rf /some/working/dir/maven/org/apache/hive
-    mvn clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven
-    mvn test -Dmaven.repo.local=/some/working/dir/maven -Dtest=nothing
+    mvn -B clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven
+    mvn -B test -Dmaven.repo.local=/some/working/dir/maven -Dtest=TestDummy
     cd itests
-    mvn clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven
-    mvn test -Dmaven.repo.local=/some/working/dir/maven -Dtest=nothing
+    mvn -B clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven
+    mvn -B test -Dmaven.repo.local=/some/working/dir/maven -Dtest=TestDummy
   elif [[ "${buildTool}" == "ant" ]]
   then
     ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy \

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=1537980&r1=1537979&r2=1537980&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 Fri Nov  1 16:34:56 2013
@@ -84,11 +84,11 @@ cd /some/working/dir/
   if [[ "ant" == "maven" ]]
   then
     rm -rf /some/working/dir/maven/org/apache/hive
-    mvn clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven
-    mvn test -Dmaven.repo.local=/some/working/dir/maven -Dtest=nothing
+    mvn -B clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven
+    mvn -B test -Dmaven.repo.local=/some/working/dir/maven -Dtest=TestDummy
     cd itests
-    mvn clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven
-    mvn test -Dmaven.repo.local=/some/working/dir/maven -Dtest=nothing
+    mvn -B clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven
+    mvn -B test -Dmaven.repo.local=/some/working/dir/maven -Dtest=TestDummy
   elif [[ "ant" == "ant" ]]
   then
     ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy \

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=1537980&r1=1537979&r2=1537980&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 Fri Nov  1 16:34:56 2013
@@ -84,11 +84,11 @@ cd /some/working/dir/
   if [[ "maven" == "maven" ]]
   then
     rm -rf /some/working/dir/maven/org/apache/hive
-    mvn clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven
-    mvn test -Dmaven.repo.local=/some/working/dir/maven -Dtest=nothing
+    mvn -B clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven
+    mvn -B test -Dmaven.repo.local=/some/working/dir/maven -Dtest=TestDummy
     cd itests
-    mvn clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven
-    mvn test -Dmaven.repo.local=/some/working/dir/maven -Dtest=nothing
+    mvn -B clean install -DskipTests -Dmaven.repo.local=/some/working/dir/maven
+    mvn -B test -Dmaven.repo.local=/some/working/dir/maven -Dtest=TestDummy
   elif [[ "maven" == "ant" ]]
   then
     ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy \

Added: hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/ssh/TestRSyncCommandExecutor.java
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/ssh/TestRSyncCommandExecutor.java?rev=1537980&view=auto
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/ssh/TestRSyncCommandExecutor.java (added)
+++ hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/ssh/TestRSyncCommandExecutor.java Fri Nov  1 16:34:56 2013
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hive.ptest.execution.ssh;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import junit.framework.Assert;
+
+import org.apache.hive.ptest.execution.Constants;
+import org.apache.hive.ptest.execution.LocalCommand;
+import org.apache.hive.ptest.execution.MockLocalCommandFactory;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestRSyncCommandExecutor {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestRSyncCommandExecutor.class);
+
+  private MockLocalCommandFactory localCommandFactory;
+
+  @Before
+  public void setup() throws Exception {
+    localCommandFactory = new MockLocalCommandFactory(LOG);
+  }
+
+  @Test
+  public void testShutdownBeforeWaitFor() throws Exception {
+    LocalCommand localCommand = mock(LocalCommand.class);
+    localCommandFactory.setInstance(localCommand);
+    RSyncCommandExecutor executor = new RSyncCommandExecutor(LOG, localCommandFactory);
+    Assert.assertFalse(executor.isShutdown());
+    executor.shutdownNow();
+    RSyncCommand command = new RSyncCommand(executor, "privateKey", "user", "host", 1, "local", "remote", RSyncCommand.Type.FROM_LOCAL);
+    executor.execute(command);
+    Assert.assertTrue(executor.isShutdown());
+    Assert.assertEquals(Constants.EXIT_CODE_UNKNOWN, command.getExitCode());
+    if(command.getException() != null) {
+      throw new Exception("Unexpected exception during execution", command.getException());
+    }
+    verify(localCommand, times(1)).kill();
+  }
+  @Test
+  public void testShutdownDuringWaitFor() throws Exception {
+    LocalCommand localCommand = mock(LocalCommand.class);
+    localCommandFactory.setInstance(localCommand);
+    final RSyncCommandExecutor executor = new RSyncCommandExecutor(LOG, localCommandFactory);
+    Assert.assertFalse(executor.isShutdown());
+    when(localCommand.getExitCode()).thenAnswer(new Answer<Integer>() {
+      @Override
+      public Integer answer(InvocationOnMock invocation) throws Throwable {
+        executor.shutdownNow();
+        return Constants.EXIT_CODE_UNKNOWN;
+      }
+    });
+    RSyncCommand command = new RSyncCommand(executor, "privateKey", "user", "host", 1, "local", "remote", RSyncCommand.Type.FROM_LOCAL);
+    executor.execute(command);
+    Assert.assertTrue(executor.isShutdown());
+    Assert.assertEquals(Constants.EXIT_CODE_UNKNOWN, command.getExitCode());
+    if(command.getException() != null) {
+      throw new Exception("Unexpected exception during execution", command.getException());
+    }
+    verify(localCommand, never()).kill();
+  }
+}
\ No newline at end of file

Added: 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=1537980&view=auto
==============================================================================
--- hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/ssh/TestSSHCommandExecutor.java (added)
+++ hive/trunk/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/ssh/TestSSHCommandExecutor.java Fri Nov  1 16:34:56 2013
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hive.ptest.execution.ssh;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import junit.framework.Assert;
+
+import org.apache.hive.ptest.execution.Constants;
+import org.apache.hive.ptest.execution.LocalCommand;
+import org.apache.hive.ptest.execution.MockLocalCommandFactory;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestSSHCommandExecutor {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestSSHCommandExecutor.class);
+
+  private MockLocalCommandFactory localCommandFactory;
+
+  @Before
+  public void setup() throws Exception {
+    localCommandFactory = new MockLocalCommandFactory(LOG);
+  }
+
+  @Test
+  public void testShutdownBeforeWaitFor() throws Exception {
+    LocalCommand localCommand = mock(LocalCommand.class);
+    localCommandFactory.setInstance(localCommand);
+    SSHCommandExecutor executor = new SSHCommandExecutor(LOG, localCommandFactory);
+    Assert.assertFalse(executor.isShutdown());
+    executor.shutdownNow();
+    SSHCommand command = new SSHCommand(executor, "privateKey", "user", "host", 1, "whoami");
+    executor.execute(command);
+    Assert.assertTrue(executor.isShutdown());
+    Assert.assertEquals(Constants.EXIT_CODE_UNKNOWN, command.getExitCode());
+    if(command.getException() != null) {
+      throw new Exception("Unexpected exception during execution", command.getException());
+    }
+    verify(localCommand, times(1)).kill();
+  }
+  @Test
+  public void testShutdownDuringWaitFor() throws Exception {
+    LocalCommand localCommand = mock(LocalCommand.class);
+    localCommandFactory.setInstance(localCommand);
+    final SSHCommandExecutor executor = new SSHCommandExecutor(LOG, localCommandFactory);
+    Assert.assertFalse(executor.isShutdown());
+    when(localCommand.getExitCode()).thenAnswer(new Answer<Integer>() {
+      @Override
+      public Integer answer(InvocationOnMock invocation) throws Throwable {
+        executor.shutdownNow();
+        return Constants.EXIT_CODE_UNKNOWN;
+      }
+    });
+    SSHCommand command = new SSHCommand(executor, "privateKey", "user", "host", 1, "whoami");
+    executor.execute(command);
+    Assert.assertTrue(executor.isShutdown());
+    Assert.assertEquals(Constants.EXIT_CODE_UNKNOWN, command.getExitCode());
+    if(command.getException() != null) {
+      throw new Exception("Unexpected exception during execution", command.getException());
+    }
+    verify(localCommand, never()).kill();
+  }
+}
\ No newline at end of file