You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by st...@apache.org on 2014/10/31 15:25:09 UTC

[01/13] git commit: SLIDER-570 tests to simulate launch failures and validate output

Repository: incubator-slider
Updated Branches:
  refs/heads/develop e23e680fe -> 2ac84287c


SLIDER-570 tests to simulate launch failures and validate output


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/f63ac6c9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/f63ac6c9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/f63ac6c9

Branch: refs/heads/develop
Commit: f63ac6c90b95386715f8487a7b810a7e26f561d7
Parents: 69712cb
Author: Steve Loughran <st...@apache.org>
Authored: Thu Oct 30 11:25:30 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Oct 31 11:07:49 2014 +0000

----------------------------------------------------------------------
 .../funtest/framework/CommandTestBase.groovy    | 24 +++++++++---------
 .../lifecycle/AgentClusterLifecycleIT.groovy    |  9 ++++++-
 .../lifecycle/AgentLaunchFailureIT.groovy       | 26 +++++++++++---------
 .../lifecycle/ClusterBuildDestroyIT.groovy      |  1 -
 4 files changed, 35 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f63ac6c9/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
index 7928642..1c65394 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
@@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.slider.api.StatusKeys
 import org.apache.slider.common.tools.ConfigHelper
-import org.apache.slider.core.exceptions.SliderException
 import org.apache.slider.core.launch.SerializedApplicationReport
 import org.apache.slider.core.main.ServiceLauncher
 import org.apache.slider.common.SliderKeys
@@ -626,17 +625,19 @@ abstract class CommandTestBase extends SliderTestUtils {
       String appTemplate,
       String resourceTemplate,
       List<String> extraArgs = [],
-      File launchReport = null) {
+      File launchReportFile = null) {
 
-    if (!launchReport) {
-      launchReport = createAppReportFile()
+    if (!launchReportFile) {
+      launchReportFile = createAppReportFile()
     }
+    // delete any previous copy of the file
+    launchReportFile.delete();
     
     List<String> commands = [
         ACTION_CREATE, name,
         ARG_TEMPLATE, appTemplate,
         ARG_RESOURCES, resourceTemplate,
-        ARG_OUTPUT, launchReport.absolutePath,
+        ARG_OUTPUT, launchReportFile.absolutePath,
         ARG_WAIT, Integer.toString(THAW_WAIT_TIME)
     ]
 
@@ -655,18 +656,17 @@ abstract class CommandTestBase extends SliderTestUtils {
     commands.addAll(extraArgs)
     SliderShell shell = new SliderShell(commands)
     shell.execute()
-    if (!shell.execute()) {
+    if (0 != shell.execute()) {
       // app has failed.
 
       // grab the app report of the last known instance of this app
       // which may not be there if it was a config failure; may be out of date
       // from a previous run
-      log.error(
-          "Launch failed with exit code ${shell.ret}")
+      log.error("Launch failed with exit code ${shell.ret}")
       shell.dumpOutput()
 
       // now grab that app report if it is there
-      def appReport = maybeLookupFromLaunchReport(launchReport)
+      def appReport = maybeLookupFromLaunchReport(launchReportFile)
       String extraText = ""
       if (appReport) {
         log.error("Application report:\n$appReport")
@@ -729,15 +729,15 @@ abstract class CommandTestBase extends SliderTestUtils {
     File reportFile = createAppReportFile();
     try {
       def shell = lookup(id, reportFile)
-      if (shell.ret) {
+      if (shell.ret == 0) {
         return maybeLoadAppReport(reportFile)
       } else {
-        log.warn("Lookup operation failed:\n" + shell.dumpOutput())
+        log.warn("Lookup operation failed with ${shell.ret}")
+        shell.dumpOutput()
         return null
       }
     } finally {
       reportFile.delete()
-      
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f63ac6c9/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
index f0fa546..8b123c3 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
@@ -66,11 +66,18 @@ public class AgentClusterLifecycleIT extends AgentCommandTestBase
 
     def clusterpath = buildClusterPath(CLUSTER)
     assert !clusterFS.exists(clusterpath)
+    File launchReportFile = createAppReportFile();
     SliderShell shell = createTemplatedSliderApplication(CLUSTER,
         APP_TEMPLATE,
-        APP_RESOURCE2)
+        APP_RESOURCE2,
+        [],
+        launchReportFile)
 
     logShell(shell)
+    assert launchReportFile.exists()
+    assert launchReportFile.size() > 0
+    def launchReport = maybeLoadAppReport(launchReportFile)
+    assert launchReport;
 
     ensureApplicationIsUp(CLUSTER)
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f63ac6c9/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
index ce1e0f1..1a0d2c3 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
@@ -20,12 +20,8 @@ package org.apache.slider.funtest.lifecycle
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.hadoop.registry.client.binding.RegistryUtils
-import org.apache.hadoop.registry.client.types.Endpoint
-import org.apache.hadoop.registry.client.types.ServiceRecord
 import org.apache.slider.api.InternalKeys
 import org.apache.slider.common.SliderExitCodes
-import org.apache.slider.common.SliderKeys
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
 import org.apache.slider.funtest.framework.AgentCommandTestBase
@@ -35,8 +31,6 @@ import org.junit.After
 import org.junit.Before
 import org.junit.Test
 
-import static org.apache.slider.core.registry.info.CustomRegistryConstants.*
-
 @CompileStatic
 @Slf4j
 public class AgentLaunchFailureIT extends AgentCommandTestBase
@@ -62,22 +56,32 @@ public class AgentLaunchFailureIT extends AgentCommandTestBase
   public void testAgentLaunchFailure() throws Throwable {
     describe("Create a failing cluster and validate failure logic")
 
+    // verify no cluster
+    assert 0 != exists(CLUSTER).ret
+ 
     // create an AM which fails to launch within a second
     File launchReportFile = createAppReportFile();
     SliderShell shell = createTemplatedSliderApplication(CLUSTER,
         APP_TEMPLATE,
         APP_RESOURCE2,
         [
-            ARG_INTERNAL, InternalKeys.CHAOS_MONKEY_ENABLED, "true",
-            ARG_INTERNAL, InternalKeys.CHAOS_MONKEY_INTERVAL_SECONDS, "1",
-            ARG_INTERNAL, InternalKeys.CHAOS_MONKEY_PROBABILITY_AM_FAILURE, "100",
+            ARG_OPTION, InternalKeys.CHAOS_MONKEY_ENABLED, "true",
+            ARG_OPTION, InternalKeys.CHAOS_MONKEY_DELAY_SECONDS, "1",
+            ARG_OPTION, InternalKeys.CHAOS_MONKEY_INTERVAL_SECONDS, "60",
+            ARG_OPTION, InternalKeys.CHAOS_MONKEY_PROBABILITY_AM_FAILURE, "100",
         ],
         launchReportFile)
 
-    maybeLookupFromLaunchReport(launchReportFile)
+    shell.dumpOutput();
+    assert launchReportFile.exists()
+    assert launchReportFile.size() > 0
+    def launchReport = maybeLoadAppReport(launchReportFile)
+    assert launchReport;
+    assert launchReport.applicationId;
+    def report = maybeLookupFromLaunchReport(launchReportFile)
+    assert report;
     ensureApplicationIsUp(CLUSTER)
 
-
     //stop
     freeze(0, CLUSTER,
         [

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f63ac6c9/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/ClusterBuildDestroyIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/ClusterBuildDestroyIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/ClusterBuildDestroyIT.groovy
index 0581ed0..f03fb63 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/ClusterBuildDestroyIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/ClusterBuildDestroyIT.groovy
@@ -68,7 +68,6 @@ public class ClusterBuildDestroyIT extends AgentCommandTestBase
             ARG_RESOURCES, APP_RESOURCE
         ])
 
-
     assert clusterFS.exists(clusterDirPath)
     //cluster exists if you don't want it to be live
     exists(EXIT_SUCCESS, CLUSTER, false)


[13/13] git commit: Merge branch 'feature/SLIDER-570-failure-text-in-funtests' into develop

Posted by st...@apache.org.
Merge branch 'feature/SLIDER-570-failure-text-in-funtests' into develop


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/2ac84287
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/2ac84287
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/2ac84287

Branch: refs/heads/develop
Commit: 2ac84287c44b9190ef7e5979a406a125df424f0c
Parents: e23e680 1f98074
Author: Steve Loughran <st...@apache.org>
Authored: Fri Oct 31 14:24:54 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Oct 31 14:24:54 2014 +0000

----------------------------------------------------------------------
 .../org/apache/slider/api/InternalKeys.java     |  27 +-
 .../org/apache/slider/client/SliderClient.java  | 166 ++++++---
 .../slider/client/SliderYarnClientImpl.java     |  18 +-
 .../org/apache/slider/common/Constants.java     |   1 -
 .../slider/common/params/ActionCreateArgs.java  |   7 +
 .../slider/common/params/ActionDestroyArgs.java |   1 +
 .../slider/common/params/ActionExistsArgs.java  |   8 +-
 .../slider/common/params/ActionLookupArgs.java  |  76 ++++
 .../slider/common/params/ActionThawArgs.java    |   8 +
 .../apache/slider/common/params/Arguments.java  |  10 +-
 .../apache/slider/common/params/ClientArgs.java |  21 +-
 .../common/params/LaunchArgsAccessor.java       |   4 +
 .../common/params/LaunchArgsDelegate.java       |  11 +
 .../common/params/SliderAMCreateAction.java     |   7 +
 .../slider/common/params/SliderActions.java     |   3 +
 .../apache/slider/common/tools/SliderUtils.java | 128 +++++--
 .../apache/slider/core/conf/MapOperations.java  |  16 +-
 .../slider/core/exceptions/ErrorStrings.java    |   2 +-
 .../slider/core/launch/AppMasterLauncher.java   |  49 ++-
 .../slider/core/launch/LaunchedApplication.java |   6 +
 .../launch/SerializedApplicationReport.java     |  92 +++++
 .../core/persist/ApplicationReportSerDeser.java |  57 +++
 .../slider/core/persist/JsonSerDeser.java       |  25 +-
 .../server/appmaster/SliderAppMaster.java       |  46 ++-
 .../server/appmaster/monkey/ChaosEntry.java     |   9 +-
 .../appmaster/monkey/ChaosMonkeyService.java    |  47 ++-
 .../TestFreezeThawFlexStandaloneAM.groovy       |  52 ++-
 .../slider/core/conf/TestConfTreeResolve.groovy |  22 +-
 .../model/monkey/TestMockMonkey.groovy          |  13 +-
 .../slider/core/conf/examples/internal.json     |   5 +-
 .../framework/AgentCommandTestBase.groovy       |  16 +-
 .../funtest/framework/CommandTestBase.groovy    | 344 ++++++++++++++++---
 .../slider/funtest/framework/SliderShell.groovy |   5 +-
 .../funtest/lifecycle/AMFailuresIT.groovy       |  19 +-
 .../lifecycle/AgentClusterLifecycleIT.groovy    |  39 ++-
 .../funtest/lifecycle/AgentFailures2IT.groovy   |  24 +-
 .../funtest/lifecycle/AgentFailuresIT.groovy    |  22 +-
 .../lifecycle/AgentLaunchFailureIT.groovy       |  98 ++++++
 .../funtest/lifecycle/AgentRegistryIT.groovy    |  22 +-
 .../lifecycle/AppsThroughAgentDemo.groovy       | 167 +++++++++
 .../funtest/lifecycle/AppsThroughAgentIT.groovy |  35 +-
 .../AppsThroughAgentQueueAndLabelsIT.groovy     |  28 +-
 .../lifecycle/ClusterBuildDestroyIT.groovy      |   1 -
 43 files changed, 1448 insertions(+), 309 deletions(-)
----------------------------------------------------------------------



[04/13] git commit: SLIDER-579 "slider lookup" command

Posted by st...@apache.org.
SLIDER-579 "slider lookup" command


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/69712cb5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/69712cb5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/69712cb5

Branch: refs/heads/develop
Commit: 69712cb5e043fe3565ebcc29f6e656f844808da5
Parents: 7346265
Author: Steve Loughran <st...@apache.org>
Authored: Thu Oct 30 11:24:58 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Oct 31 11:07:49 2014 +0000

----------------------------------------------------------------------
 .../src/main/java/org/apache/slider/client/SliderClient.java     | 1 +
 .../main/java/org/apache/slider/common/params/ClientArgs.java    | 1 +
 .../java/org/apache/slider/core/exceptions/ErrorStrings.java     | 2 +-
 .../main/java/org/apache/slider/core/persist/JsonSerDeser.java   | 4 +++-
 4 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/69712cb5/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index f1f8f97..b2e343d 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -1019,6 +1019,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       ApplicationReport report =
           launchedApplication.getApplicationReport();
       SerializedApplicationReport sar = new SerializedApplicationReport(report);
+      sar.submitTime = System.currentTimeMillis();
       ApplicationReportSerDeser serDeser = new ApplicationReportSerDeser();
       serDeser.save(sar, launchArgs.getOutputFile());
     }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/69712cb5/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
index 5ee6cd5..a3b7997 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
@@ -96,6 +96,7 @@ public class ClientArgs extends CommonArgs {
         actionInstallKeytabArgs,
         actionKillContainerArgs,
         actionListArgs,
+        actionLookupArgs,
         actionRegistryArgs,
         actionResolveArgs,
         actionStatusArgs,

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/69712cb5/slider-core/src/main/java/org/apache/slider/core/exceptions/ErrorStrings.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/exceptions/ErrorStrings.java b/slider-core/src/main/java/org/apache/slider/core/exceptions/ErrorStrings.java
index 894f19b..8b04969 100644
--- a/slider-core/src/main/java/org/apache/slider/core/exceptions/ErrorStrings.java
+++ b/slider-core/src/main/java/org/apache/slider/core/exceptions/ErrorStrings.java
@@ -20,7 +20,7 @@ package org.apache.slider.core.exceptions;
 
 public interface ErrorStrings {
   String E_UNSTABLE_CLUSTER = "Unstable Application Instance :";
-  String E_CLUSTER_RUNNING = "Application Instance already running";
+  String E_CLUSTER_RUNNING = "Application Instance running";
   String E_ALREADY_EXISTS = "already exists";
   String PRINTF_E_INSTANCE_ALREADY_EXISTS = "Application Instance \"%s\" already exists and is defined in %s";
   String PRINTF_E_INSTANCE_DIR_ALREADY_EXISTS = "Application Instance dir already exists: %s";

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/69712cb5/slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java b/slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java
index 2c119ff..4d7d310 100644
--- a/slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java
+++ b/slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java
@@ -244,8 +244,10 @@ public class JsonSerDeser<T> {
       String json = toJson(instance);
       byte[] b = json.getBytes(UTF_8);
       dataOutputStream.write(b);
-    } finally {
+      dataOutputStream.flush();
       dataOutputStream.close();
+    } finally {
+      IOUtils.closeStream(dataOutputStream);
     }
   }
 


[03/13] git commit: SLIDER-570: failure output in failed tests...making sure most recent runs get reported, even failures

Posted by st...@apache.org.
SLIDER-570: failure output in failed tests...making sure most recent runs get reported, even failures


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/09d16a13
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/09d16a13
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/09d16a13

Branch: refs/heads/develop
Commit: 09d16a130c3ccb4f2f0195ca23d2fa53074e138b
Parents: e23e680
Author: Steve Loughran <st...@apache.org>
Authored: Tue Oct 28 15:48:23 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Oct 31 11:07:49 2014 +0000

----------------------------------------------------------------------
 .../org/apache/slider/client/SliderClient.java  |  70 +++++-----
 .../slider/client/SliderYarnClientImpl.java     |   7 +-
 .../org/apache/slider/common/Constants.java     |   1 -
 .../apache/slider/common/tools/SliderUtils.java | 128 ++++++++++++++-----
 .../framework/AgentCommandTestBase.groovy       |   6 +-
 .../funtest/framework/CommandTestBase.groovy    |  18 ++-
 6 files changed, 159 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/09d16a13/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index 06efadf..392280a 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -161,7 +161,6 @@ import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Properties;
 import java.util.Set;
 import java.util.regex.Pattern;
 
@@ -193,8 +192,6 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   private SliderYarnClientImpl yarnClient;
   private YarnAppListClient YarnAppListClient;
   private AggregateConf launchedInstanceDefinition;
-//  private SliderRegistryService registry;
-
 
   /**
    * The YARN registry service
@@ -1009,7 +1006,13 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
                         serviceArgs.isDebug());
     applicationId = launchedApplication.getApplicationId();
 
-    return waitForAppAccepted(launchedApplication, launchArgs.getWaittime());
+    int waittime = launchArgs.getWaittime();
+    if (waittime > 0) {
+      return waitForAppRunning(launchedApplication, waittime, waittime);
+    } else {
+      // no waiting
+      return EXIT_SUCCESS;
+    }
   }
 
   /**
@@ -1384,34 +1387,38 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   
   
   /**
-   * Wait for the launched app to be accepted
-   * @param waittime time in millis
-   * @return exit code
+   * Wait for the launched app to be accepted in the time  
+   * and, optionally running.
+   * <p>
+   * If the application
+   *
+   * @param launchedApplication application
+   * @param acceptWaitMillis time in millis to wait for accept
+   * @param runWaitMillis time in millis to wait for the app to be running.
+   * May be null, in which case no wait takes place
+   * @return exit code: success
    * @throws YarnException
    * @throws IOException
    */
-  public int waitForAppAccepted(LaunchedApplication launchedApplication, 
-                                int waittime) throws
-                                              YarnException,
-                                              IOException {
+  public int waitForAppRunning(LaunchedApplication launchedApplication,
+      int acceptWaitMillis, int runWaitMillis) throws YarnException, IOException {
     assert launchedApplication != null;
     int exitCode;
     // wait for the submit state to be reached
     ApplicationReport report = launchedApplication.monitorAppToState(
       YarnApplicationState.ACCEPTED,
-      new Duration(Constants.ACCEPT_TIME));
-
+      new Duration(acceptWaitMillis));
 
     // may have failed, so check that
     if (SliderUtils.hasAppFinished(report)) {
       exitCode = buildExitCode(report);
     } else {
       // exit unless there is a wait
-      exitCode = EXIT_SUCCESS;
 
-      if (waittime != 0) {
+
+      if (runWaitMillis != 0) {
         // waiting for state to change
-        Duration duration = new Duration(waittime * 1000);
+        Duration duration = new Duration(runWaitMillis * 1000);
         duration.start();
         report = launchedApplication.monitorAppToState(
           YarnApplicationState.RUNNING, duration);
@@ -1419,10 +1426,10 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
             report.getYarnApplicationState() == YarnApplicationState.RUNNING) {
           exitCode = EXIT_SUCCESS;
         } else {
-
-          launchedApplication.kill("");
           exitCode = buildExitCode(report);
         }
+      } else {
+        exitCode = EXIT_SUCCESS;
       }
     }
     return exitCode;
@@ -1579,16 +1586,17 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   }
 
   /**
-   * Build an exit code for an application Id and its report.
-   * If the report parameter is null, the app is killed
-   * @param report report
+   * Build an exit code for an application from its report.
+   * If the report parameter is null, its interpreted as a timeout
+   * @param report report application report
    * @return the exit code
+   * @throws IOException
+   * @throws YarnException
    */
   private int buildExitCode(ApplicationReport report) throws
                                                       IOException,
                                                       YarnException {
     if (null == report) {
-      forceKillApplication("Reached client specified timeout for application");
       return EXIT_TIMED_OUT;
     }
 
@@ -1615,6 +1623,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
         log.info("Application Failed. YarnState={}, DSFinalStatus={}", state,
                  dsStatus);
         return EXIT_YARN_SERVICE_FAILED;
+
       default:
         //not in any of these states
         return EXIT_SUCCESS;
@@ -1730,7 +1739,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     
     // and those the RM knows about
     List<ApplicationReport> instances = listSliderInstances(null);
-    SliderUtils.sortApplicationReport(instances);
+    SliderUtils.sortApplicationsByMostRecent(instances);
     Map<String, ApplicationReport> reportMap =
         SliderUtils.buildApplicationReportMap(instances, min, max);
     log.debug("Persisted {} deployed {} filtered[{}-{}] & de-duped to {}",
@@ -1746,6 +1755,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       if (persistent == null) {
         throw unknownClusterException(clustername);
       }
+      // create a new map with only that instance in it.
+      // this restricts the output of results to this instance
       persistentInstances = new HashMap<String, Path>();
       persistentInstances.put(clustername, persistent);  
     }
@@ -1767,6 +1778,13 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return listed > 0 ? EXIT_SUCCESS: EXIT_FALSE;
   }
 
+  /**
+   * Convert the instance details of an application to a string
+   * @param name instance name
+   * @param report the application report
+   * @param verbose verbose output
+   * @return a string
+   */
   String instanceDetailsToString(String name,
       ApplicationReport report,
       boolean verbose) {
@@ -1994,12 +2012,6 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return YarnAppListClient.findAllLiveInstances(appname);
   }
 
-
-  public ApplicationReport findClusterInInstanceList(List<ApplicationReport> instances,
-                                                     String appname) {
-    return yarnClient.findClusterInInstanceList(instances, appname);
-  }
-
   /**
    * Connect to a Slider AM
    * @param app application report providing the details on the application

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/09d16a13/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java b/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
index a00f3b2..2f18b7a 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
@@ -252,9 +252,10 @@ public class SliderYarnClientImpl extends YarnClientImpl {
 
   /**
    * Find a cluster in the instance list; biased towards live instances
-   * @param instances
-   * @param appname
-   * @return
+   * @param instances list of instances
+   * @param appname application name
+   * @return the first found instance, else a failed/finished instance, or null
+   * if there are none of those
    */
   public ApplicationReport findClusterInInstanceList(List<ApplicationReport> instances,
                                                      String appname) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/09d16a13/slider-core/src/main/java/org/apache/slider/common/Constants.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/Constants.java b/slider-core/src/main/java/org/apache/slider/common/Constants.java
index 2fe0250..868ea57 100644
--- a/slider-core/src/main/java/org/apache/slider/common/Constants.java
+++ b/slider-core/src/main/java/org/apache/slider/common/Constants.java
@@ -19,7 +19,6 @@
 package org.apache.slider.common;
 
 public class Constants {
-  public static final int ACCEPT_TIME = 60000;
   public static final int CONNECT_TIMEOUT = 10000;
   public static final int RPC_TIMEOUT = 15000;
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/09d16a13/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
index 0f622c9..ee8693f 100644
--- a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
+++ b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
@@ -76,6 +76,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InterruptedIOException;
 import java.io.PrintWriter;
+import java.io.Serializable;
 import java.io.StringWriter;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
@@ -84,6 +85,8 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.net.URLDecoder;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -97,6 +100,7 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
+import java.util.TimeZone;
 import java.util.Timer;
 import java.util.TimerTask;
 import java.util.TreeMap;
@@ -599,36 +603,55 @@ public final class SliderUtils {
         builder.append(tag).append(separator);
       }
     }
+    DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm");
+    dateFormat.setTimeZone(TimeZone.getDefault());
     builder.append("state: ").append(r.getYarnApplicationState());
-    builder.append(separator).append("URL: ").append(r.getTrackingUrl());
+    String trackingUrl = r.getTrackingUrl();
+    if (isSet(trackingUrl)) {
+      builder.append(separator).append("URL: ").append(trackingUrl);
+    }
     builder.append(separator)
-           .append("Started ")
-           .append(new Date(r.getStartTime()).toGMTString());
+           .append("Started: ")
+           .append(dateFormat.format(new Date(r.getStartTime())));
     long finishTime = r.getFinishTime();
     if (finishTime > 0) {
       builder.append(separator)
-             .append("Finished ")
-             .append(new Date(finishTime).toGMTString());
+             .append("Finished: ")
+             .append(dateFormat.format(new Date(finishTime)));
+    }
+    String rpcHost = r.getHost();
+    if (!isSet(rpcHost)) {
+      builder.append(separator)
+             .append("RPC :")
+             .append(rpcHost)
+             .append(':')
+             .append(r.getRpcPort());
     }
-    builder.append(separator)
-           .append("RPC :")
-           .append(r.getHost())
-           .append(':')
-           .append(r.getRpcPort());
     String diagnostics = r.getDiagnostics();
-    if (!diagnostics.isEmpty()) {
+    if (!isSet(diagnostics)) {
       builder.append(separator).append("Diagnostics :").append(diagnostics);
     }
     return builder.toString();
   }
 
+
+  /**
+   * Sorts the given list of application reports, most recently started 
+   * or finished instance first.
+   *
+   * @param instances list of instances
+   */
+  public static void sortApplicationsByMostRecent(List<ApplicationReport> instances) {
+    Collections.sort(instances, new MostRecentlyStartedOrFinishedFirst());
+  }
+
   /**
    * Sorts the given list of application reports
    * Finished instances are ordered by finished time and running/accepted instances are
    * ordered by start time
    * Finally Instance are order by finished instances coming after running instances
    *
-   * @param instances list of intances
+   * @param instances list of instances
    */
   public static void sortApplicationReport(List<ApplicationReport> instances) {
     if (instances.size() <= 1) {
@@ -650,28 +673,10 @@ public final class SliderUtils {
     }
 
     if (liveInstance.size() > 1) {
-      Comparator<ApplicationReport> liveInstanceComparator =
-          new Comparator<ApplicationReport>() {
-            @Override
-            public int compare(ApplicationReport r1, ApplicationReport r2) {
-              long x = r1.getStartTime();
-              long y = r2.getStartTime();
-              return (x < y) ? -1 : ((x == y) ? 0 : 1);
-            }
-          };
-      Collections.sort(liveInstance, liveInstanceComparator);
+      Collections.sort(liveInstance, new MostRecentlyStartedAppFirst());
     }
     if (nonLiveInstance.size() > 1) {
-      Comparator<ApplicationReport> nonLiveInstanceComparator =
-          new Comparator<ApplicationReport>() {
-            @Override
-            public int compare(ApplicationReport r1, ApplicationReport r2) {
-              long x = r1.getFinishTime();
-              long y = r2.getFinishTime();
-              return (x < y) ? -1 : ((x == y) ? 0 : 1);
-            }
-          };
-      Collections.sort(nonLiveInstance, nonLiveInstanceComparator);
+      Collections.sort(nonLiveInstance, new MostRecentAppFinishFirst());
     }
     instances.clear();
     instances.addAll(liveInstance);
@@ -2067,4 +2072,63 @@ public final class SliderUtils {
     }
     return result;
   }
+
+  /**
+   * Compare the times of two applications: most recent app comes first
+   * Specifically: the one whose start time value is greater.
+   */
+  private static class MostRecentlyStartedAppFirst
+      implements Comparator<ApplicationReport>, Serializable {
+    @Override
+    public int compare(ApplicationReport r1, ApplicationReport r2) {
+      long x = r1.getStartTime();
+      long y = r2.getStartTime();
+      return compareTwoLongs(x, y);
+    }
+  }
+  
+  /**
+   * Compare the times of two applications: most recent app comes first.
+   * "Recent"== the app whose start time <i>or finish time</i> is the greatest.
+   */
+  private static class MostRecentlyStartedOrFinishedFirst
+      implements Comparator<ApplicationReport>, Serializable {
+    @Override
+    public int compare(ApplicationReport r1, ApplicationReport r2) {
+      long started1 = r1.getStartTime();
+      long started2 = r2.getStartTime();
+      long finished1 = r1.getFinishTime();
+      long finished2 = r2.getFinishTime();
+      long lastEvent1 = Math.max(started1, finished1);
+      long lastEvent2 = Math.max(started2, finished2);
+      return compareTwoLongs(lastEvent1, lastEvent2);
+    }
+  }
+
+  /**
+   * Compare the times of two applications: most recently finished app comes first
+   * Specifically: the one whose finish time value is greater.
+   */
+  private static class MostRecentAppFinishFirst
+      implements Comparator<ApplicationReport>, Serializable {
+    @Override
+    public int compare(ApplicationReport r1, ApplicationReport r2) {
+      long x = r1.getFinishTime();
+      long y = r2.getFinishTime();
+      return compareTwoLongs(x, y);
+    }
+  }
+
+  /**
+   * Compare two long values for sorting. As the return value for 
+   * comparators must be int, the simple value of <code>x-y</code>
+   * is inapplicable
+   * @param x x value
+   * @param y y value
+   * @return -ve if x is less than y, +ve if y is greater than x; 0 for equality
+   */
+  public static int compareTwoLongs(long x, long y) {
+    return (x < y) ? -1 : ((x == y) ? 0 : 1);
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/09d16a13/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentCommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentCommandTestBase.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentCommandTestBase.groovy
index b44ae07..6c99ab6 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentCommandTestBase.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentCommandTestBase.groovy
@@ -111,9 +111,9 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
       SliderShell shell = slider(EXIT_SUCCESS,
           [
               ACTION_INSTALL_PACKAGE,
-              Arguments.ARG_NAME, TEST_APP_PKG_NAME,
-              Arguments.ARG_PACKAGE, zipFileName.absolutePath,
-              Arguments.ARG_REPLACE_PKG
+              ARG_NAME, TEST_APP_PKG_NAME,
+              ARG_PACKAGE, zipFileName.absolutePath,
+              ARG_REPLACE_PKG
           ])
       logShell(shell)
       log.info "App pkg uploaded at home directory .slider/package/$TEST_APP_PKG_NAME/$TEST_APP_PKG_FILE"

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/09d16a13/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
index f3d91d3..7b50c60 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
@@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.slider.api.StatusKeys
 import org.apache.slider.common.tools.ConfigHelper
-import org.apache.slider.core.main.LauncherExitCodes
 import org.apache.slider.core.main.ServiceLauncher
 import org.apache.slider.common.SliderKeys
 import org.apache.slider.common.SliderXmlConfKeys
@@ -615,7 +614,6 @@ abstract class CommandTestBase extends SliderTestUtils {
         ARG_TEMPLATE, appTemplate,
         ARG_RESOURCES, resourceTemplate,
         ARG_WAIT, Integer.toString(THAW_WAIT_TIME)
-        
     ]
 
     maybeAddCommandOption(commands,
@@ -631,7 +629,21 @@ abstract class CommandTestBase extends SliderTestUtils {
         [ARG_COMP_OPT, SliderKeys.COMPONENT_AM, SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL],
         SLIDER_CONFIG.getTrimmed(SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL));
     commands.addAll(extraArgs)
-    SliderShell shell = slider(LauncherExitCodes.EXIT_SUCCESS, commands)
+    SliderShell shell = new SliderShell(commands)
+    shell.execute()
+    if (!shell.execute()) {
+      // app has failed.
+      
+      // grab the app report of the last known instance of this app
+      // which may not be there if it was a config failure; may be out of date
+      // from a previous run
+      log.error("Launch failed with exit code ${shell.ret}.\nLast instance of $name:")
+      slider([ACTION_LIST, name, ARG_VERBOSE]).dumpOutput()
+      
+      // trigger the assertion failure
+      shell.assertExitCode(EXIT_SUCCESS)
+    }
+    
     return shell
   }
 


[12/13] git commit: SLIDER-570: AgentClusterLifecycleIT -> lookup/appId driven on thaw operations

Posted by st...@apache.org.
SLIDER-570: AgentClusterLifecycleIT -> lookup/appId driven on thaw operations


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

Branch: refs/heads/develop
Commit: 1f98074f16e7840d11727d9bcf1efb5a7858595e
Parents: e1023ea
Author: Steve Loughran <st...@apache.org>
Authored: Fri Oct 31 14:24:44 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Oct 31 14:24:44 2014 +0000

----------------------------------------------------------------------
 .../funtest/framework/CommandTestBase.groovy    |  1 +
 .../lifecycle/AgentClusterLifecycleIT.groovy    | 26 +++++++++++++++++---
 2 files changed, 23 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1f98074f/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
index 7ea84df..072b07e 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
@@ -692,6 +692,7 @@ abstract class CommandTestBase extends SliderTestUtils {
         "launch",
         ".json",
         new File("target"))
+    reportFile.delete()
     return reportFile
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1f98074f/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
index cb137ce..a0cb6ca 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
@@ -20,6 +20,7 @@ package org.apache.slider.funtest.lifecycle
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.slider.api.ClusterDescription
 import org.apache.slider.api.StatusKeys
 import org.apache.slider.client.SliderClient
@@ -65,6 +66,7 @@ public class AgentClusterLifecycleIT extends AgentCommandTestBase
 
     def clusterpath = buildClusterPath(CLUSTER)
     assert !clusterFS.exists(clusterpath)
+
     File launchReportFile = createAppReportFile();
     SliderShell shell = createTemplatedSliderApplication(CLUSTER,
         APP_TEMPLATE,
@@ -138,6 +140,9 @@ public class AgentClusterLifecycleIT extends AgentCommandTestBase
           ARG_MESSAGE, "freeze-in-test-cluster-lifecycle"
       ])
       describe " >>> Cluster is now frozen."
+      
+      // should be in finished state, as this was a clean shutdown
+      assertInYarnState(appId, YarnApplicationState.FINISHED)
 
       //cluster exists if you don't want it to be live
       exists(0, CLUSTER, false)
@@ -152,12 +157,19 @@ public class AgentClusterLifecycleIT extends AgentCommandTestBase
       list(-1, [ARG_LIVE])
       list(-1, [ARG_STATE, "running"])
       list( 0, [ARG_STATE, "FINISHED"])
-      
+
+      def thawReport = createAppReportFile()
       //start then stop the cluster
       thaw(CLUSTER,
           [
               ARG_WAIT, Integer.toString(THAW_WAIT_TIME),
+              ARG_OUTPUT, thawReport.absolutePath,
           ])
+      def thawedAppId = ensureYarnApplicationIsUp(thawReport)
+     
+
+      assertAppRunning(thawedAppId)
+
       exists(0, CLUSTER)
       describe " >>> Cluster is now thawed."
       list(0, [CLUSTER, ARG_LIVE])
@@ -175,6 +187,8 @@ public class AgentClusterLifecycleIT extends AgentCommandTestBase
 
       describe " >>> Cluster is now force frozen - 2nd time."
 
+      // new instance should be in killed state
+      assertInYarnState(thawedAppId, YarnApplicationState.KILLED)
       //cluster is no longer live
       exists(0, CLUSTER, false)
 
@@ -183,19 +197,23 @@ public class AgentClusterLifecycleIT extends AgentCommandTestBase
 
       //start with a restart count set to enable restart
       describe "the kill/restart phase may fail if yarn.resourcemanager.am.max-attempts is too low"
-      
+
+      def thawReport2 = createAppReportFile()
+      //start then stop the cluster
       thaw(CLUSTER,
           [
               ARG_WAIT, Integer.toString(THAW_WAIT_TIME),
-              ARG_DEFINE, SliderXmlConfKeys.KEY_AM_RESTART_LIMIT + "=3"
+              ARG_DEFINE, SliderXmlConfKeys.KEY_AM_RESTART_LIMIT + "=3",
+              ARG_OUTPUT, thawReport2.absolutePath
           ])
-
+      def thawedAppId2 = ensureYarnApplicationIsUp(thawReport2)
       describe " >>> Cluster is now thawed - 2nd time."
 
 
       describe " >>> Kill AM and wait for restart."
       ClusterDescription status = killAmAndWaitForRestart(sliderClient, CLUSTER)
 
+      assertAppRunning(thawedAppId2)
       def restarted = status.getInfo(
           StatusKeys.INFO_CONTAINERS_AM_RESTART)
       assert restarted != null


[05/13] git commit: SLIDER-572 add startup delay for the chaos monkey

Posted by st...@apache.org.
SLIDER-572 add startup delay for the chaos monkey


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/517042f2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/517042f2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/517042f2

Branch: refs/heads/develop
Commit: 517042f2e6878445610884eaa1b47fa3a349ed17
Parents: 25bbfeb
Author: Steve Loughran <st...@apache.org>
Authored: Wed Oct 29 20:46:32 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Oct 31 11:07:49 2014 +0000

----------------------------------------------------------------------
 .../org/apache/slider/api/InternalKeys.java     | 11 ++++++++-
 .../apache/slider/core/conf/MapOperations.java  |  2 +-
 .../server/appmaster/SliderAppMaster.java       | 24 ++++++++++----------
 .../appmaster/monkey/ChaosMonkeyService.java    | 20 +++++++++-------
 .../model/monkey/TestMockMonkey.groovy          |  4 ++--
 5 files changed, 37 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/517042f2/slider-core/src/main/java/org/apache/slider/api/InternalKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/api/InternalKeys.java b/slider-core/src/main/java/org/apache/slider/api/InternalKeys.java
index 5f150e6..074644d 100644
--- a/slider-core/src/main/java/org/apache/slider/api/InternalKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/api/InternalKeys.java
@@ -121,6 +121,14 @@ public interface InternalKeys {
   int DEFAULT_CHAOS_MONKEY_INTERVAL_HOURS = 0;
   int DEFAULT_CHAOS_MONKEY_INTERVAL_MINUTES = 0;
 
+  String CHAOS_MONKEY_DELAY = "internal.chaos.monkey.delay";
+  String CHAOS_MONKEY_DELAY_DAYS = CHAOS_MONKEY_DELAY + ".days";
+  String CHAOS_MONKEY_DELAY_HOURS = CHAOS_MONKEY_DELAY + ".hours";
+  String CHAOS_MONKEY_DELAY_MINUTES = CHAOS_MONKEY_DELAY + ".minutes";
+  String CHAOS_MONKEY_DELAY_SECONDS = CHAOS_MONKEY_DELAY + ".seconds";
+  
+  int DEFAULT_CHAOS_MONKEY_STARTUP_DELAY = 0;
+
   /**
    * Prefix for all chaos monkey probabilities
    */
@@ -133,7 +141,8 @@ public interface InternalKeys {
   /**
    * Probability of a monkey check killing the AM:  {@value}
    */
-  String CHAOS_MONKEY_PROBABILITY_AM_FAILURE = CHAOS_MONKEY_PROBABILITY +".amfailure";
+  String CHAOS_MONKEY_PROBABILITY_AM_FAILURE =
+      CHAOS_MONKEY_PROBABILITY + ".amfailure";
 
   /**
    * Default probability of a monkey check killing the AM:  {@value}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/517042f2/slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java b/slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java
index 6503c9b..de8fc2c 100644
--- a/slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java
+++ b/slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java
@@ -282,7 +282,7 @@ public class MapOperations implements Map<String, String> {
    * @param defHours
    * @param defMins
    * @param defSecs
-   * @return
+   * @return the aggregate time range in seconds
    */
   public long getTimeRange(String basekey,
       int defDays,

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/517042f2/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index 39a2572..855ecd7 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -1101,14 +1101,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
   }
 
-/*
-
-  @Override
-  protected RegistryOperationsService createRegistryOperationsInstance() {
-    return new ResourceManagerRegistryService("YarnRegistry");
-  }
-*/
-
   /**
    * TODO: purge this once RM is doing the work
    * @throws IOException
@@ -1133,7 +1125,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     if (instance == null) {
       return false;
     }
-    // this is where component registrations will go
+    // this is where component registrations  go
     log.info("Registering component {}", id);
     String cid = RegistryPathUtils.encodeYarnID(id.toString());
     ServiceRecord container = new ServiceRecord();
@@ -2185,8 +2177,16 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
           "Chaos monkey not configured with a time interval...not enabling");
       return false;
     }
-    log.info("Adding Chaos Monkey scheduled every {} seconds ({} hours)",
-        monkeyInterval, monkeyInterval/(60*60));
+
+    long monkeyDelay = internals.getTimeRange(
+        InternalKeys.CHAOS_MONKEY_DELAY,
+        0,
+        0,
+        0,
+        (int)monkeyInterval);
+    
+    log.info("Adding Chaos Monkey scheduled every {} seconds ({} hours -delay {}",
+        monkeyInterval, monkeyInterval/(60*60), monkeyDelay);
     monkey = new ChaosMonkeyService(metrics, actionQueues);
     initAndAddService(monkey);
     
@@ -2204,7 +2204,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
         containerKillProbability);
     
     // and schedule it
-    if (monkey.schedule(monkeyInterval, TimeUnit.SECONDS)) {
+    if (monkey.schedule(monkeyDelay, monkeyInterval, TimeUnit.SECONDS)) {
       log.info("Chaos Monkey is running");
       return true;
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/517042f2/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java
index f7b1eb7..80f981c 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java
@@ -92,13 +92,14 @@ public class ChaosMonkeyService extends AbstractService {
 
   /**
    * Schedule the monkey
-   * @param time interval
+   *
+   * @param delay initial delay
    * @param timeUnit time unit
    * @return true if it was scheduled (i.e. 1+ action) and interval > 0
    */
-  public boolean schedule(long time, TimeUnit timeUnit) {
-    if (time > 0 && !chaosEntries.isEmpty()) {
-      queues.schedule(getChaosAction(time, timeUnit));
+  public boolean schedule(long delay, long interval, TimeUnit timeUnit) {
+    if (interval > 0 && !chaosEntries.isEmpty()) {
+      queues.schedule(getChaosAction(delay, interval, timeUnit));
       return true;
     } else {
       return false;
@@ -107,15 +108,18 @@ public class ChaosMonkeyService extends AbstractService {
 
   /**
    * Get the chaos action
-   * @param time interval
+   *
+   * @param delay
    * @param timeUnit time unit
    * @return the action to schedule
    */
-  public RenewingAction<MonkeyPlayAction> getChaosAction(long time, TimeUnit timeUnit) {
+  public RenewingAction<MonkeyPlayAction> getChaosAction(long delay,
+      long interval,
+      TimeUnit timeUnit) {
     RenewingAction<MonkeyPlayAction> action = new RenewingAction<MonkeyPlayAction>(
         new MonkeyPlayAction(this, 0, TimeUnit.MILLISECONDS),
-        time,
-        time,
+        delay,
+        interval,
         timeUnit,
         0
     );

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/517042f2/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/monkey/TestMockMonkey.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/monkey/TestMockMonkey.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/monkey/TestMockMonkey.groovy
index 26d07ee..d31c9f6 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/monkey/TestMockMonkey.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/monkey/TestMockMonkey.groovy
@@ -78,7 +78,7 @@ class TestMockMonkey extends BaseMockAppStateTest {
     assert 0 == monkey.targetCount;
     monkey.addTarget("target", counter, ChaosMonkeyService.PERCENT_100)
     assert 1 == monkey.targetCount;
-    assert monkey.schedule(1, TimeUnit.SECONDS)
+    assert monkey.schedule(0, 1, TimeUnit.SECONDS)
     assert 1 == queues.scheduledActions.size()
   }
 
@@ -96,7 +96,7 @@ class TestMockMonkey extends BaseMockAppStateTest {
   public void testMonkeyScheduleProb0Actions() throws Throwable {
     ChaosCounter counter = new ChaosCounter()
     monkey.addTarget("target", counter, 0)
-    assert !monkey.schedule(1, TimeUnit.SECONDS)
+    assert !monkey.schedule(0, 1, TimeUnit.SECONDS)
     assert 0 == queues.scheduledActions.size()
   }
 


[02/13] git commit: SLIDER-570 sort found clusters by recent-ness

Posted by st...@apache.org.
SLIDER-570 sort found clusters by recent-ness


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/7b630000
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/7b630000
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/7b630000

Branch: refs/heads/develop
Commit: 7b63000063609f9db0408e27b0b9356474f29ebc
Parents: 09d16a1
Author: Steve Loughran <st...@apache.org>
Authored: Wed Oct 29 11:44:37 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Oct 31 11:07:49 2014 +0000

----------------------------------------------------------------------
 .../org/apache/slider/client/SliderClient.java  | 21 +++++++++++---------
 .../slider/client/SliderYarnClientImpl.java     | 11 +++++-----
 2 files changed, 17 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/7b630000/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index 392280a..6ebbcdd 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -1693,13 +1693,16 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return actionList(clustername, args);
   }
 
-    /**
-     * Implement the list action: list all nodes
-  
-     * @param clustername List out specific instance name
-     * @param args Action list arguments
-     * @return 0 if one or more entries were listed
-     */
+  /**
+   * Implement the list action.
+   * @param clustername List out specific instance name
+   * @param args Action list arguments
+   * @return 0 if one or more entries were listed
+   * @throws IOException
+   * @throws YarnException
+   * @throws UnknownApplicationInstanceException if a specific instance
+   * was named but it was not found
+   */
   @Override
   @VisibleForTesting
   public int actionList(String clustername, ActionListArgs args)
@@ -1996,8 +1999,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   private RunningApplication findApplication(String appname)
       throws YarnException, IOException {
     ApplicationReport applicationReport = findInstance(appname);
-    return applicationReport != null ? new RunningApplication(yarnClient, applicationReport): null; 
-      
+    return applicationReport != null ?
+           new RunningApplication(yarnClient, applicationReport): null; 
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/7b630000/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java b/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
index 2f18b7a..a2a7fe7 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
@@ -259,19 +259,18 @@ public class SliderYarnClientImpl extends YarnClientImpl {
    */
   public ApplicationReport findClusterInInstanceList(List<ApplicationReport> instances,
                                                      String appname) {
+    // sort by most recent
+    SliderUtils.sortApplicationsByMostRecent(instances);
     ApplicationReport found = null;
-    ApplicationReport foundAndLive = null;
     for (ApplicationReport app : instances) {
       if (app.getName().equals(appname)) {
-        found = app;
         if (isApplicationLive(app)) {
-          foundAndLive = app;
+          return app;
         }
+        // set the found value if not set
+        found = found != null ? found : app;
       }
     }
-    if (foundAndLive != null) {
-      found = foundAndLive;
-    }
     return found;
   }
 


[06/13] git commit: SLIDER-579 Add CLI operations/--out args to get the app reports of launched apps

Posted by st...@apache.org.
SLIDER-579 Add CLI operations/--out args to get the app reports of launched apps


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/25bbfebb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/25bbfebb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/25bbfebb

Branch: refs/heads/develop
Commit: 25bbfebb052c59c74950225db759ee93f2f181ba
Parents: 7b63000
Author: Steve Loughran <st...@apache.org>
Authored: Wed Oct 29 14:59:00 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Oct 31 11:07:49 2014 +0000

----------------------------------------------------------------------
 .../org/apache/slider/client/SliderClient.java  | 54 +++++++++--
 .../slider/common/params/ActionCreateArgs.java  |  7 ++
 .../slider/common/params/ActionExistsArgs.java  |  8 +-
 .../slider/common/params/ActionLookupArgs.java  | 76 +++++++++++++++
 .../slider/common/params/ActionThawArgs.java    |  8 ++
 .../apache/slider/common/params/Arguments.java  | 10 +-
 .../apache/slider/common/params/ClientArgs.java | 20 +++-
 .../common/params/LaunchArgsAccessor.java       |  4 +
 .../common/params/LaunchArgsDelegate.java       | 11 +++
 .../common/params/SliderAMCreateAction.java     |  7 ++
 .../slider/common/params/SliderActions.java     |  3 +
 .../slider/core/launch/AppMasterLauncher.java   | 49 +++++++---
 .../slider/core/launch/LaunchedApplication.java |  6 ++
 .../launch/SerializedApplicationReport.java     | 97 ++++++++++++++++++++
 .../core/persist/ApplicationReportSerDeser.java | 57 ++++++++++++
 .../slider/core/persist/JsonSerDeser.java       | 21 ++++-
 .../TestFreezeThawFlexStandaloneAM.groovy       | 52 ++++++++++-
 17 files changed, 457 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index 6ebbcdd..f1f8f97 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -45,6 +45,8 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
+import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.registry.client.api.RegistryOperations;
 
@@ -55,6 +57,7 @@ import org.apache.hadoop.registry.client.exceptions.NoRecordException;
 import org.apache.hadoop.registry.client.types.Endpoint;
 import org.apache.hadoop.registry.client.types.ServiceRecord;
 import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.slider.api.ClusterDescription;
 import org.apache.slider.api.ClusterNode;
 import org.apache.slider.api.InternalKeys;
@@ -78,6 +81,7 @@ import org.apache.slider.common.params.ActionFlexArgs;
 import org.apache.slider.common.params.ActionFreezeArgs;
 import org.apache.slider.common.params.ActionKillContainerArgs;
 import org.apache.slider.common.params.ActionListArgs;
+import org.apache.slider.common.params.ActionLookupArgs;
 import org.apache.slider.common.params.ActionRegistryArgs;
 import org.apache.slider.common.params.ActionResolveArgs;
 import org.apache.slider.common.params.ActionStatusArgs;
@@ -114,7 +118,9 @@ import org.apache.slider.core.launch.CommandLineBuilder;
 import org.apache.slider.core.launch.JavaCommandLineBuilder;
 import org.apache.slider.core.launch.LaunchedApplication;
 import org.apache.slider.core.launch.RunningApplication;
+import org.apache.slider.core.launch.SerializedApplicationReport;
 import org.apache.slider.core.main.RunService;
+import org.apache.slider.core.persist.ApplicationReportSerDeser;
 import org.apache.slider.core.persist.ConfPersister;
 import org.apache.slider.core.persist.LockAcquireFailedException;
 import org.apache.slider.core.registry.SliderRegistryUtils;
@@ -389,6 +395,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
           serviceArgs.getActionAMSuicideArgs());
     } else if (ACTION_LIST.equals(action)) {
       exitCode = actionList(clusterName, serviceArgs.getActionListArgs());
+    } else if (ACTION_LOOKUP.equals(action)) {
+      exitCode = actionLookup(serviceArgs.getActionLookupArgs());
     } else if (ACTION_REGISTRY.equals(action)) {
       exitCode = actionRegistry(serviceArgs.getActionRegistryArgs());
     } else if (ACTION_RESOLVE.equals(action)) {
@@ -1006,6 +1014,14 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
                         serviceArgs.isDebug());
     applicationId = launchedApplication.getApplicationId();
 
+    if (launchArgs.getOutputFile() != null) {
+      // output file has been requested. Get the app report and serialize it
+      ApplicationReport report =
+          launchedApplication.getApplicationReport();
+      SerializedApplicationReport sar = new SerializedApplicationReport(report);
+      ApplicationReportSerDeser serDeser = new ApplicationReportSerDeser();
+      serDeser.save(sar, launchArgs.getOutputFile());
+    }
     int waittime = launchArgs.getWaittime();
     if (waittime > 0) {
       return waitForAppRunning(launchedApplication, waittime, waittime);
@@ -1374,12 +1390,6 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       amLauncher.setQueue(amQueue);
     }
 
-    // Submit the application to the applications manager
-    // SubmitApplicationResponse submitResp = applicationsManager.submitApplication(appRequest);
-    // Ignore the response as either a valid response object is returned on success
-    // or an exception thrown to denote some form of a failure
-    
-
     // submit the application
     LaunchedApplication launchedApplication = amLauncher.submitApplication();
     return launchedApplication;
@@ -3300,6 +3310,38 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     print("\n");
   }
 
+  /**
+   * Implement the lookup action.
+   * @param args Action arguments
+   * @return 0 if the entry was found
+   * @throws IOException
+   * @throws YarnException
+   * @throws UnknownApplicationInstanceException if a specific instance
+   * was named but it was not found
+   */
+  @VisibleForTesting
+  public int actionLookup(ActionLookupArgs args)
+      throws IOException, YarnException {
+    verifyBindingsDefined();
+    try {
+      ApplicationId id = ConverterUtils.toApplicationId(args.id);
+      ApplicationReport report = yarnClient.getApplicationReport(id);
+      SerializedApplicationReport sar = new SerializedApplicationReport(report);
+      ApplicationReportSerDeser serDeser = new ApplicationReportSerDeser();
+      if (args.outputFile != null) {
+        serDeser.save(sar, args.outputFile);
+      } else {
+        println(serDeser.toJson(sar));
+      }
+    } catch (IllegalArgumentException e) {
+      throw new BadCommandArgumentsException(e, "%s : %s", args, e);
+    } catch (ApplicationAttemptNotFoundException notFound) {
+      throw new NotFoundException(notFound, notFound.toString());
+    } catch (ApplicationNotFoundException notFound) {
+      throw new NotFoundException(notFound, notFound.toString());
+    }
+    return EXIT_SUCCESS;
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/common/params/ActionCreateArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/ActionCreateArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/ActionCreateArgs.java
index cfcfb9d..e70f30a 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/ActionCreateArgs.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/ActionCreateArgs.java
@@ -21,6 +21,8 @@ package org.apache.slider.common.params;
 import com.beust.jcommander.Parameters;
 import com.beust.jcommander.ParametersDelegate;
 
+import java.io.File;
+
 @Parameters(commandNames = {SliderActions.ACTION_CREATE},
             commandDescription = SliderActions.DESCRIBE_ACTION_CREATE)
 
@@ -36,6 +38,11 @@ public class ActionCreateArgs extends AbstractClusterBuildingActionArgs
   LaunchArgsDelegate launchArgs = new LaunchArgsDelegate();
 
   @Override
+  public File getOutputFile() {
+    return launchArgs.getOutputFile();
+  }
+
+  @Override
   public String getRmAddress() {
     return launchArgs.getRmAddress();
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/common/params/ActionExistsArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/ActionExistsArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/ActionExistsArgs.java
index 047d32c..dd1c04b 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/ActionExistsArgs.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/ActionExistsArgs.java
@@ -21,6 +21,8 @@ package org.apache.slider.common.params;
 import com.beust.jcommander.Parameter;
 import com.beust.jcommander.Parameters;
 
+import java.io.File;
+
 @Parameters(commandNames = {SliderActions.ACTION_EXISTS},
             commandDescription = SliderActions.DESCRIBE_ACTION_EXISTS)
 
@@ -30,6 +32,7 @@ public class ActionExistsArgs extends AbstractActionArgs {
   public String getActionName() {
     return SliderActions.ACTION_EXISTS;
   }
+
   @Parameter(names = {ARG_LIVE},
              description = "verify that the application is running")
   public boolean live;
@@ -37,5 +40,8 @@ public class ActionExistsArgs extends AbstractActionArgs {
   @Parameter(names = {ARG_STATE},
              description = "verify that the application is in the specific YARN state")
   public String state = "";
-  
+
+  @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT},
+      description = "output file for any application report")
+  public File out;
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/common/params/ActionLookupArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/ActionLookupArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/ActionLookupArgs.java
new file mode 100644
index 0000000..3b69e74
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/common/params/ActionLookupArgs.java
@@ -0,0 +1,76 @@
+/*
+ * 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.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import org.apache.commons.lang.StringUtils;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.UsageException;
+
+import java.io.File;
+
+@Parameters(commandNames = {SliderActions.ACTION_LOOKUP},
+            commandDescription = SliderActions.DESCRIBE_ACTION_LOOKUP)
+
+public class ActionLookupArgs extends AbstractActionArgs {
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_LOOKUP;
+  }
+
+  public int getMinParams() {
+    return 0;
+  }
+  public int getMaxParams() {
+    return 0;
+  }
+  
+  @Parameter(names = {ARG_ID},
+             description = "ID of the container")
+  public String id;
+
+  @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT},
+      description = "output file for any application report")
+  public File outputFile;
+
+  @Override
+  public void validate() throws BadCommandArgumentsException, UsageException {
+    super.validate();
+    if (StringUtils.isEmpty(id)) {
+      throw new BadCommandArgumentsException("Missing mandatory argument "
+                                             + ARG_ID);
+    }
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder(SliderActions.ACTION_LOOKUP);
+    if (id!=null) {
+      sb.append(" ");
+      sb.append(ARG_ID).append(" ").append(id);
+    }
+    if (outputFile != null) {
+      sb.append(" ");
+      sb.append(ARG_OUTPUT).append(" ").append(outputFile.getAbsolutePath());
+    }
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java
index 8408385..b43a14e 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java
@@ -21,6 +21,8 @@ package org.apache.slider.common.params;
 import com.beust.jcommander.Parameters;
 import com.beust.jcommander.ParametersDelegate;
 
+import java.io.File;
+
 @Parameters(commandNames = {SliderActions.ACTION_THAW},
             commandDescription = SliderActions.DESCRIBE_ACTION_THAW)
 public class ActionThawArgs extends AbstractActionArgs implements
@@ -50,4 +52,10 @@ public class ActionThawArgs extends AbstractActionArgs implements
   public void setWaittime(int waittime) {
     launchArgs.setWaittime(waittime);
   }
+
+
+  @Override
+  public File getOutputFile() {
+    return launchArgs.getOutputFile();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java b/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
index 06d9dfb..42efb33 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
@@ -49,8 +49,9 @@ public interface Arguments {
    */
   String ARG_FILESYSTEM = "--fs";
   String ARG_FILESYSTEM_LONG = "--filesystem";
-  String ARG_FORMAT = "--format";
+  String ARG_FOLDER = "--folder";
   String ARG_FORCE = "--force";
+  String ARG_FORMAT = "--format";
   String ARG_GETCONF = "--getconf";
   String ARG_GETEXP = "--getexp";
   String ARG_GETFILES = "--getfiles";
@@ -58,6 +59,7 @@ public interface Arguments {
   String ARG_ID = "--id";
   String ARG_IMAGE = "--image";
   String ARG_INTERNAL = "--internal";
+  String ARG_KEYTAB = "--keytab";
   String ARG_LEVEL = "--level";
   String ARG_LIST = "--list";
   String ARG_LISTCONF = "--listconf";
@@ -67,19 +69,17 @@ public interface Arguments {
   String ARG_MANAGER = "--manager";
   String ARG_MANAGER_SHORT = "--m";
   String ARG_MESSAGE = "--message";
+  String ARG_NAME = "--name";
   String ARG_OPTION = "--option";
   String ARG_OPTION_SHORT = "-O";
-  String ARG_NAME = "--name";
-  String ARG_FOLDER = "--folder";
   String ARG_OUTPUT = "--out";
   String ARG_OUTPUT_SHORT = "-o";
+  String ARG_OVERWRITE = "--overwrite";
   String ARG_PACKAGE = "--package";
-  String ARG_KEYTAB = "--keytab";
   String ARG_PATH = "--path";
   String ARG_PROVIDER = "--provider";
   String ARG_QUEUE = "--queue";
   String ARG_REPLACE_PKG = "--replacepkg";
-  String ARG_OVERWRITE = "--overwrite";
   String ARG_RESOURCES = "--resources";
   String ARG_RES_COMP_OPT = "--rescompopt";
   String ARG_RES_COMP_OPT_SHORT = "--rco";

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
index 5a1c049..5ee6cd5 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
@@ -43,28 +43,31 @@ public class ClientArgs extends CommonArgs {
    * This is not bonded to jcommander, it is set up
    * after the construction to point to the relevant
    * entry
+   * 
+   * KEEP IN ALPHABETICAL ORDER
    */
   private AbstractClusterBuildingActionArgs buildingActionArgs;
   private final ActionAMSuicideArgs actionAMSuicideArgs = new ActionAMSuicideArgs();
   private final ActionBuildArgs actionBuildArgs = new ActionBuildArgs();
-  private final ActionInstallPackageArgs actionInstallPackageArgs = new ActionInstallPackageArgs();
-  private final ActionInstallKeytabArgs actionInstallKeytabArgs = new ActionInstallKeytabArgs();
-  private final ActionUpdateArgs actionUpdateArgs = new ActionUpdateArgs();
   private final ActionCreateArgs actionCreateArgs = new ActionCreateArgs();
   private final ActionDestroyArgs actionDestroyArgs = new ActionDestroyArgs();
+  private final ActionDiagnosticArgs actionDiagnosticArgs = new ActionDiagnosticArgs();
   private final ActionExistsArgs actionExistsArgs = new ActionExistsArgs();
   private final ActionFlexArgs actionFlexArgs = new ActionFlexArgs();
   private final ActionFreezeArgs actionFreezeArgs = new ActionFreezeArgs();
+  private final ActionHelpArgs actionHelpArgs = new ActionHelpArgs();
+  private final ActionInstallPackageArgs actionInstallPackageArgs = new ActionInstallPackageArgs();
+  private final ActionInstallKeytabArgs actionInstallKeytabArgs = new ActionInstallKeytabArgs();
   private final ActionKillContainerArgs actionKillContainerArgs =
     new ActionKillContainerArgs();
   private final ActionListArgs actionListArgs = new ActionListArgs();
+  private final ActionLookupArgs actionLookupArgs = new ActionLookupArgs();
   private final ActionRegistryArgs actionRegistryArgs = new ActionRegistryArgs();
   private final ActionResolveArgs actionResolveArgs = new ActionResolveArgs();
   private final ActionStatusArgs actionStatusArgs = new ActionStatusArgs();
   private final ActionThawArgs actionThawArgs = new ActionThawArgs();
+  private final ActionUpdateArgs actionUpdateArgs = new ActionUpdateArgs();
   private final ActionVersionArgs actionVersionArgs = new ActionVersionArgs();
-  private final ActionHelpArgs actionHelpArgs = new ActionHelpArgs();
-  private final ActionDiagnosticArgs actionDiagnosticArgs = new ActionDiagnosticArgs();
 
 
   public ClientArgs(String[] args) {
@@ -171,6 +174,10 @@ public class ClientArgs extends CommonArgs {
     return actionListArgs;
   }
 
+  public ActionLookupArgs getActionLookupArgs() {
+    return actionLookupArgs;
+  }
+
   public ActionRegistryArgs getActionRegistryArgs() {
     return actionRegistryArgs;
   }
@@ -242,6 +249,9 @@ public class ClientArgs extends CommonArgs {
     } else if (SliderActions.ACTION_LIST.equals(action)) {
       bindCoreAction(actionListArgs);
 
+    } else if (SliderActions.ACTION_LOOKUP.equals(action)) {
+      bindCoreAction(actionLookupArgs);
+
     } else if (SliderActions.ACTION_REGISTRY.equals(action)) {
       bindCoreAction(actionRegistryArgs);
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsAccessor.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsAccessor.java b/slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsAccessor.java
index c36a968..7524053 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsAccessor.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsAccessor.java
@@ -18,9 +18,13 @@
 
 package org.apache.slider.common.params;
 
+import java.io.File;
+
 /**
  * Launch args for create and start and anything else that can start something
  */
 public interface LaunchArgsAccessor extends WaitTimeAccessor {
   String getRmAddress();
+
+  File getOutputFile();
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsDelegate.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsDelegate.java b/slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsDelegate.java
index f0068e2..bc7e94c 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsDelegate.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsDelegate.java
@@ -20,6 +20,8 @@ package org.apache.slider.common.params;
 
 import com.beust.jcommander.Parameter;
 
+import java.io.File;
+
 /**
  * Any launch-time args
  */
@@ -37,4 +39,13 @@ public class LaunchArgsDelegate extends WaitArgsDelegate implements
   public String getRmAddress() {
     return rmAddress;
   }
+
+  @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT},
+      description = "output file for any application report")
+  public File outputFile;
+
+  @Override
+  public File getOutputFile() {
+    return outputFile;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/common/params/SliderAMCreateAction.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/SliderAMCreateAction.java b/slider-core/src/main/java/org/apache/slider/common/params/SliderAMCreateAction.java
index 0d084da..197c22b 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/SliderAMCreateAction.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/SliderAMCreateAction.java
@@ -22,6 +22,8 @@ import com.beust.jcommander.Parameter;
 import com.beust.jcommander.Parameters;
 import com.beust.jcommander.ParametersDelegate;
 
+import java.io.File;
+
 
 @Parameters(commandNames = {SliderActions.ACTION_CREATE},
             commandDescription = SliderActions.DESCRIBE_ACTION_CREATE)
@@ -63,5 +65,10 @@ public class SliderAMCreateAction extends AbstractActionArgs implements
   public void setWaittime(int waittime) {
     launchArgs.setWaittime(waittime);
   }
+
+  @Override
+  public File getOutputFile() {
+    return launchArgs.getOutputFile();
+  }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java b/slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java
index 87e4582..4b92a32 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java
@@ -36,6 +36,7 @@ public interface SliderActions {
   String ACTION_HELP = "help";
   String ACTION_KILL_CONTAINER = "kill-container";
   String ACTION_LIST = "list";
+  String ACTION_LOOKUP = "lookup";
   String ACTION_PREFLIGHT = "preflight";
   String ACTION_RECONFIGURE = "reconfigure";
   String ACTION_REGISTRY = "registry";
@@ -68,6 +69,8 @@ public interface SliderActions {
   String DESCRIBE_ACTION_HELP = "Print help information";
   String DESCRIBE_ACTION_LIST =
                   "List running Slider applications";
+  String DESCRIBE_ACTION_LOOKUP =
+                  "look up a YARN application";
   String DESCRIBE_ACTION_MONITOR =
                     "Monitor a running application";
   String DESCRIBE_ACTION_REGISTRY =

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/core/launch/AppMasterLauncher.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/AppMasterLauncher.java b/slider-core/src/main/java/org/apache/slider/core/launch/AppMasterLauncher.java
index 303f777..7023c80 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/AppMasterLauncher.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/AppMasterLauncher.java
@@ -46,18 +46,19 @@ public class AppMasterLauncher extends AbstractLauncher {
   private static final Logger log =
     LoggerFactory.getLogger(AppMasterLauncher.class);
 
-  protected final YarnClientApplication application;
-  private final String name;
-  private final String type;
-  private final ApplicationSubmissionContext submissionContext;
-  private final ApplicationId appId;
-  private final boolean secureCluster;
+  public final YarnClientApplication application;
+  public final String name;
+  public final String type;
+  public final ApplicationSubmissionContext submissionContext;
+  public final ApplicationId appId;
+  public final boolean secureCluster;
   private int maxAppAttempts = 0;
   private boolean keepContainersOverRestarts = true;
   private String queue = YarnConfiguration.DEFAULT_QUEUE_NAME;
   private int priority = 1;
   private final Resource resource = Records.newRecord(Resource.class);
   private final SliderYarnClientImpl yarnClient;
+  private Long submitTime;
 
   /**
    * Build the AM Launcher
@@ -65,10 +66,15 @@ public class AppMasterLauncher extends AbstractLauncher {
    * @param type applicatin type
    * @param conf hadoop config
    * @param fs filesystem binding
-   * @param application precreated YARN client app instance
+   * @param yarnClient yarn client
    * @param secureCluster is the cluster secure?
-   * @param options map of options. All values are extracted in this constructor only
    * -the map is not retained.
+   * @param secureCluster flag to indicate secure cluster
+   * @param options map of options. All values are extracted in this constructor only
+   * @param resourceGlobalOptions global options
+   * @param applicationTags any app tags
+   * @throws IOException
+   * @throws YarnException
    */
   public AppMasterLauncher(String name,
                            String type,
@@ -192,7 +198,6 @@ public class AppMasterLauncher extends AbstractLauncher {
     completeContainerLaunch();
     submissionContext.setAMContainerSpec(containerLaunchContext);
     return submissionContext;
-
   }
 
   /**
@@ -216,13 +221,35 @@ public class AppMasterLauncher extends AbstractLauncher {
     fs.addDelegationTokens(tokenRenewer, credentials);
   }
 
- 
+  /**
+   * Submit the application. 
+   * @return a launched application representing the submitted application
+   * @throws IOException
+   * @throws YarnException
+   */
   public LaunchedApplication submitApplication() throws IOException, YarnException {
     completeAppMasterLaunch();
     log.info("Submitting application to Resource Manager");
     ApplicationId applicationId =
       yarnClient.submitApplication(submissionContext);
+    // implicit success; record the time
+    submitTime = System.currentTimeMillis();
     return new LaunchedApplication(applicationId, yarnClient);
   }
-  
+
+  /**
+   * Build a serializable application report. This is a very minimal
+   * report that contains the application Id, name and type —the information
+   * available
+   * @return a data structure which can be persisted
+   */
+  public SerializedApplicationReport createSerializedApplicationReport() {
+    SerializedApplicationReport sar = new SerializedApplicationReport();
+    sar.applicationId = appId.toString();
+    sar.name = name;
+    sar.applicationType = type;
+    sar.queue = queue;
+    sar.submitTime = submitTime;
+    return sar;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/core/launch/LaunchedApplication.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/LaunchedApplication.java b/slider-core/src/main/java/org/apache/slider/core/launch/LaunchedApplication.java
index e5a025c..632e3fd 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/LaunchedApplication.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/LaunchedApplication.java
@@ -95,6 +95,12 @@ public class LaunchedApplication {
     return yarnClient.killRunningApplication(applicationId, reason);
   }
 
+  /**
+   * Get the application report of this application
+   * @return an application report
+   * @throws YarnException
+   * @throws IOException
+   */
   public ApplicationReport getApplicationReport()
     throws YarnException, IOException {
     return yarnClient.getApplicationReport(applicationId);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java b/slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java
new file mode 100644
index 0000000..e3d77d0
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java
@@ -0,0 +1,97 @@
+/*
+ * 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.slider.core.launch;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.slider.core.persist.ApplicationReportSerDeser;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.io.IOException;
+
+/**
+ * Serialized form of an application report which can be persisted
+ * and then parsed. It can not be converted back into a
+ * real YARN application report
+ * 
+ * Useful for testing
+ */
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+
+public class SerializedApplicationReport {
+
+  public String applicationId;
+  public String applicationAttemptId;
+  public String name;
+  public String applicationType;
+  public String user;
+  public String queue;
+  public String host;
+  public Integer rpcPort;
+  public String state;
+  public String diagnostics;
+  public String url;
+  /**
+   * This value is non-null only when a report is generated from a submission context.
+   * The YARN {@link ApplicationReport} structure does not propagate this value
+   * from the RM.
+   */
+  public Long submitTime;
+  public Long startTime;
+  public Long finishTime;
+  public String finalStatus;
+  public String origTrackingUrl;
+  public Float progress;
+  
+  public SerializedApplicationReport() {
+  }
+  
+  public SerializedApplicationReport(ApplicationReport report) {
+    this.applicationId = report.getApplicationId().toString();
+    this.applicationAttemptId = report.getCurrentApplicationAttemptId().toString();
+    this.name = report.getName();
+    this.applicationType = report.getApplicationType();
+    this.user = report.getUser();
+    this.queue = report.getQueue();
+    this.host = report.getHost();
+    this.rpcPort = report.getRpcPort();
+    this.state = report.getYarnApplicationState().toString();
+    this.diagnostics = report.getDiagnostics();
+    this.startTime = report.getStartTime();
+    this.finishTime = report.getFinishTime();
+    this.finalStatus = report.getFinalApplicationStatus().toString();
+    this.progress = report.getProgress();
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return ApplicationReportSerDeser.toString(this);
+    } catch (IOException e) {
+      return super.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/core/persist/ApplicationReportSerDeser.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/persist/ApplicationReportSerDeser.java b/slider-core/src/main/java/org/apache/slider/core/persist/ApplicationReportSerDeser.java
new file mode 100644
index 0000000..a8c72ce
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/core/persist/ApplicationReportSerDeser.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.persist;
+
+import org.apache.slider.core.launch.SerializedApplicationReport;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.JsonParseException;
+import org.codehaus.jackson.map.JsonMappingException;
+
+import java.io.IOException;
+
+/**
+ * Persistence of {@link SerializedApplicationReport}
+ * 
+ */
+public class ApplicationReportSerDeser
+    extends JsonSerDeser<SerializedApplicationReport> {
+  public ApplicationReportSerDeser() {
+    super(SerializedApplicationReport.class);
+  }
+
+
+  private static final ApplicationReportSerDeser
+      staticinstance = new ApplicationReportSerDeser();
+
+  /**
+   * Convert an instance to a JSON string -sync access to a shared ser/deser
+   * object instance
+   * @param instance object to convert
+   * @return a JSON string description
+   * @throws JsonParseException parse problems
+   * @throws JsonMappingException O/J mapping problems
+   */
+  public static String toString(SerializedApplicationReport instance)
+      throws IOException, JsonGenerationException, JsonMappingException {
+    synchronized (staticinstance) {
+      return staticinstance.toJson(instance);
+    }
+  }
+ 
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java b/slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java
index f77d6df..2c119ff 100644
--- a/slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java
+++ b/slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java
@@ -33,12 +33,13 @@ import org.codehaus.jackson.map.SerializationConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.DataOutputStream;
 import java.io.EOFException;
 import java.io.File;
 import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
 
 /**
  * Support for marshalling objects to and from JSON.
@@ -126,6 +127,7 @@ public class JsonSerDeser<T> {
    * @throws IOException IO problems
    * @throws JsonMappingException failure to map from the JSON to this class
    */
+  @SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
   public synchronized T fromResource(String resource)
       throws IOException, JsonParseException, JsonMappingException {
     InputStream resStream = null;
@@ -206,9 +208,10 @@ public class JsonSerDeser<T> {
 
 
   /**
-   * Save a cluster description to a hadoop filesystem
+   * Save to a hadoop filesystem
    * @param fs filesystem
    * @param path path
+   * @param instance instance to save
    * @param overwrite should any existing file be overwritten
    * @throws IOException IO exception
    */
@@ -220,13 +223,23 @@ public class JsonSerDeser<T> {
   }
 
   /**
+   * Save an instance to a file
+   * @param instance instance to save
+   * @param file file
+   * @throws IOException
+   */
+  public void save(T instance, File file) throws
+      IOException {
+    writeJsonAsBytes(instance, new FileOutputStream(file));
+  }
+  
+  /**
    * Write the json as bytes -then close the file
    * @param dataOutputStream an outout stream that will always be closed
    * @throws IOException on any failure
    */
   private void writeJsonAsBytes(T instance,
-                                DataOutputStream dataOutputStream) throws
-                                                                   IOException {
+      OutputStream dataOutputStream) throws IOException {
     try {
       String json = toJson(instance);
       byte[] b = json.getBytes(UTF_8);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/25bbfebb/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeThawFlexStandaloneAM.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeThawFlexStandaloneAM.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeThawFlexStandaloneAM.groovy
index 7e7e52c..70d5ba5 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeThawFlexStandaloneAM.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeThawFlexStandaloneAM.groovy
@@ -24,11 +24,19 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic
 import org.apache.hadoop.fs.FileSystem as HadoopFS
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.slider.agent.AgentMiniClusterTestBase
 import org.apache.slider.client.SliderClient
+import org.apache.slider.common.params.ActionLookupArgs
+import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.tools.SliderUtils
+import org.apache.slider.core.exceptions.BadCommandArgumentsException
+import org.apache.slider.core.exceptions.NotFoundException
+import org.apache.slider.core.exceptions.SliderException
+import org.apache.slider.core.main.ServiceLaunchException
 import org.apache.slider.core.main.ServiceLauncher
+import org.apache.slider.core.persist.ApplicationReportSerDeser
 import org.junit.Test
 
 /**
@@ -80,12 +88,35 @@ class TestFreezeThawFlexStandaloneAM extends AgentMiniClusterTestBase {
     localFS.delete(tempConfPath,true)
     
     //now start the cluster
-    ServiceLauncher launcher2 = thawCluster(clustername, [], true);
+    File appreport = new File("target/$clustername/appreport.json")
+    ServiceLauncher launcher2 = thawCluster(clustername,
+        [Arguments.ARG_OUTPUT, appreport.absolutePath],
+        true);
+
     SliderClient newCluster = launcher2.service
     addToTeardown(newCluster);
+    ApplicationReportSerDeser serDeser = new ApplicationReportSerDeser();
+    def sar = serDeser.fromFile(appreport)
+    log.info(sar.toString())
+    assert sar.applicationId != null
+
+    describe("lookup")
+
+    // now via lookup
+    appreport.delete()
+    def lookup1 = new ActionLookupArgs()
+    lookup1.id = sar.applicationId
+
+    assert 0 == newCluster.actionLookup(lookup1)
+    lookup1.outputFile = appreport
+    assert 0 == newCluster.actionLookup(lookup1)
+    sar = serDeser.fromFile(appreport)
+    assert sar.state == YarnApplicationState.RUNNING.toString()
+    
 
     newCluster.getClusterDescription(clustername);
     
+    describe("no change flex")
     // while running, flex it with no changes
     newCluster.flex(clustername, [:]);
 
@@ -95,9 +126,28 @@ class TestFreezeThawFlexStandaloneAM extends AgentMiniClusterTestBase {
     report = newCluster.applicationReport
     assert report.finalApplicationStatus == FinalApplicationStatus.KILLED
 
+    assert 0 == newCluster.actionLookup(lookup1)
+    sar = serDeser.fromFile(appreport)
+    assert sar.finalStatus == FinalApplicationStatus.KILLED.toString()
+    
     //stop again
     assert 0 == clusterActionFreeze(newCluster, clustername)
 
+    // and add some invalid lookup operations for
+    
+    def lookup2 = new ActionLookupArgs()
+    lookup2.id = "invalid"
+    try {
+      newCluster.actionLookup(lookup2)
+      fail("expected $lookup2 to fail")
+    } catch (BadCommandArgumentsException expected) {
+    }
+    try {
+      lookup2.id = "application_1414593568640_0002"
+      newCluster.actionLookup(lookup2)
+      fail("expected $lookup2 to fail")
+    } catch (NotFoundException expected) {
+    }
   }
 
 }


[08/13] git commit: SLIDER-570 monkey can simulate launch failure, ongoing work on tests

Posted by st...@apache.org.
SLIDER-570 monkey can simulate launch failure, ongoing work on tests


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

Branch: refs/heads/develop
Commit: 1cd37f57053933065baf864d109fba1897ee8b95
Parents: f63ac6c
Author: Steve Loughran <st...@apache.org>
Authored: Thu Oct 30 18:32:59 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Oct 31 11:07:50 2014 +0000

----------------------------------------------------------------------
 .../org/apache/slider/api/InternalKeys.java     |  16 ++
 .../org/apache/slider/client/SliderClient.java  |  20 +--
 .../slider/common/params/ActionDestroyArgs.java |   1 +
 .../launch/SerializedApplicationReport.java     |   5 -
 .../server/appmaster/SliderAppMaster.java       |  20 ++-
 .../server/appmaster/monkey/ChaosEntry.java     |   9 +-
 .../appmaster/monkey/ChaosMonkeyService.java    |  27 ++--
 .../model/monkey/TestMockMonkey.groovy          |   9 +-
 .../funtest/framework/CommandTestBase.groovy    |  55 +++++--
 .../slider/funtest/framework/SliderShell.groovy |   1 -
 .../funtest/lifecycle/AMFailuresIT.groovy       |   4 +-
 .../lifecycle/AgentClusterLifecycleIT.groovy    |   3 +-
 .../lifecycle/AgentLaunchFailureIT.groovy       |  12 +-
 .../lifecycle/AppsThroughAgentDemo.groovy       | 160 +++++++++++++++++++
 .../funtest/lifecycle/AppsThroughAgentIT.groovy |   8 +-
 15 files changed, 293 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1cd37f57/slider-core/src/main/java/org/apache/slider/api/InternalKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/api/InternalKeys.java b/slider-core/src/main/java/org/apache/slider/api/InternalKeys.java
index 074644d..b360fbe 100644
--- a/slider-core/src/main/java/org/apache/slider/api/InternalKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/api/InternalKeys.java
@@ -150,6 +150,12 @@ public interface InternalKeys {
   int DEFAULT_CHAOS_MONKEY_PROBABILITY_AM_FAILURE = 0;
 
   /**
+   * Probability of a monkey check killing the AM:  {@value}
+   */
+  String CHAOS_MONKEY_PROBABILITY_AM_LAUNCH_FAILURE =
+      CHAOS_MONKEY_PROBABILITY + ".amlaunchfailure";
+
+  /**
    * Probability of a monkey check killing a container:  {@value}
    */
 
@@ -162,4 +168,14 @@ public interface InternalKeys {
   int DEFAULT_CHAOS_MONKEY_PROBABILITY_CONTAINER_FAILURE = 0;
 
 
+  /**
+   * 1% of chaos
+   */
+  int PROBABILITY_PERCENT_1 = 100;
+  
+  /**
+   * 100% for chaos values
+   */
+  int PROBABILITY_PERCENT_100 = 100 * PROBABILITY_PERCENT_1;
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1cd37f57/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index b2e343d..0e87906 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -347,7 +347,6 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     } catch (PathNotFoundException nfe) {
       throw new NotFoundException(nfe, nfe.toString());
     }
-
   }
 
   /**
@@ -529,7 +528,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     SliderUtils.validateClusterName(clustername);
     //no=op, it is now mandatory. 
     verifyBindingsDefined();
-    verifyNoLiveClusters(clustername);
+    verifyNoLiveClusters(clustername, "Destroy");
 
     // create the directory path
     Path clusterDirectory = sliderFileSystem.buildClusterDirPath(clustername);
@@ -772,7 +771,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     SliderUtils.validateClusterName(clustername);
     verifyBindingsDefined();
     if (!liveClusterAllowed) {
-      verifyNoLiveClusters(clustername);
+      verifyNoLiveClusters(clustername, "Create");
     }
 
     Configuration conf = getConfig();
@@ -1100,7 +1099,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
 
     deployedClusterName = clustername;
     SliderUtils.validateClusterName(clustername);
-    verifyNoLiveClusters(clustername);
+    verifyNoLiveClusters(clustername, "Launch");
     Configuration config = getConfig();
     lookupZKQuorum();
     boolean clusterSecure = SliderUtils.isHadoopClusterSecure(config);
@@ -1534,17 +1533,21 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   /**
    * verify that a live cluster isn't there
    * @param clustername cluster name
+   * @param action
    * @throws SliderException with exit code EXIT_CLUSTER_LIVE
    * if a cluster of that name is either live or starting up.
    */
-  public void verifyNoLiveClusters(String clustername) throws
+  public void verifyNoLiveClusters(String clustername, String action) throws
                                                        IOException,
                                                        YarnException {
     List<ApplicationReport> existing = findAllLiveInstances(clustername);
 
     if (!existing.isEmpty()) {
       throw new SliderException(EXIT_APPLICATION_IN_USE,
-                              clustername + ": " + E_CLUSTER_RUNNING + " :" +
+          action +" failed for "
+                              + clustername
+                              + ": "
+                              + E_CLUSTER_RUNNING + " :" +
                               existing.get(0));
     }
   }
@@ -2168,10 +2171,9 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   @Override
   public int actionThaw(String clustername, ActionThawArgs thaw) throws YarnException, IOException {
     SliderUtils.validateClusterName(clustername);
-    // see if it is actually running and bail out;
     verifyBindingsDefined();
-    verifyNoLiveClusters(clustername);
-
+    // see if it is actually running and bail out;
+    verifyNoLiveClusters(clustername, "Start");
 
     //start the cluster
     return startCluster(clustername, thaw);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1cd37f57/slider-core/src/main/java/org/apache/slider/common/params/ActionDestroyArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/ActionDestroyArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/ActionDestroyArgs.java
index d4acea6..1203d28 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/ActionDestroyArgs.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/ActionDestroyArgs.java
@@ -24,6 +24,7 @@ import com.beust.jcommander.Parameters;
             commandDescription = SliderActions.DESCRIBE_ACTION_DESTROY)
 
 public class ActionDestroyArgs extends AbstractActionArgs {
+  
   @Override
   public String getActionName() {
     return SliderActions.ACTION_DESTROY;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1cd37f57/slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java b/slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java
index e3d77d0..dfa037d 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java
@@ -18,12 +18,7 @@
 
 package org.apache.slider.core.launch;
 
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.slider.core.persist.ApplicationReportSerDeser;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1cd37f57/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index 855ecd7..5e2ba11 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -192,7 +192,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     ServiceStateChangeListener,
     RoleKeys,
     ProviderCompleted {
-  
+
   protected static final Logger log =
     LoggerFactory.getLogger(SliderAppMaster.class);
 
@@ -220,7 +220,9 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
    * Singleton of metrics registry
    */
   public static final MetricRegistry metrics = new MetricRegistry();
-  
+  public static final String E_TRIGGERED_LAUNCH_FAILURE =
+      "Chaos monkey triggered launch failure";
+
   /** YARN RPC to communicate with the Resource Manager or Node Manager */
   private YarnRPC yarnRPC;
 
@@ -2191,6 +2193,20 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     initAndAddService(monkey);
     
     // configure the targets
+    
+    // launch failure: special case with explicit failure triggered now
+    int amLaunchFailProbability = internals.getOptionInt(
+        InternalKeys.CHAOS_MONKEY_PROBABILITY_AM_LAUNCH_FAILURE,
+        0);
+    if (amLaunchFailProbability> 0 && monkey.chaosCheck(amLaunchFailProbability)) {
+      // trigger a failure
+      ActionStopSlider stop = new ActionStopSlider("stop",
+          0, TimeUnit.SECONDS,
+          LauncherExitCodes.EXIT_FALSE,
+          FinalApplicationStatus.FAILED,
+          E_TRIGGERED_LAUNCH_FAILURE);
+    }
+    
     int amKillProbability = internals.getOptionInt(
         InternalKeys.CHAOS_MONKEY_PROBABILITY_AM_FAILURE,
         InternalKeys.DEFAULT_CHAOS_MONKEY_PROBABILITY_AM_FAILURE);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1cd37f57/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java
index 2869fe9..87a0aaa 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java
@@ -22,6 +22,7 @@ import com.codahale.metrics.Counter;
 import com.codahale.metrics.MetricRegistry;
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang.StringUtils;
+import org.apache.slider.api.InternalKeys;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -42,16 +43,16 @@ public class ChaosEntry {
 
   /**
    * Constructor -includes validation of all arguments
-   * @param name
-   * @param target
-   * @param probability
+   * @param name entry name
+   * @param target target
+   * @param probability probability of occurring
    */
   public ChaosEntry(String name, ChaosTarget target, long probability,
       MetricRegistry metrics) {
     Preconditions.checkArgument(!StringUtils.isEmpty(name), "missing name");
     Preconditions.checkArgument(target != null, "null target");
     Preconditions.checkArgument(probability > 0, "negative probability");
-    Preconditions.checkArgument(probability <= ChaosMonkeyService.PERCENT_100,
+    Preconditions.checkArgument(probability <= InternalKeys.PROBABILITY_PERCENT_100,
         "probability over 100%: "+ probability);
     this.name = name;
     this.target = target;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1cd37f57/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java
index 80f981c..27219e4 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java
@@ -20,6 +20,7 @@ package org.apache.slider.server.appmaster.monkey;
 
 import com.codahale.metrics.MetricRegistry;
 import org.apache.hadoop.service.AbstractService;
+import org.apache.slider.api.InternalKeys;
 import org.apache.slider.server.appmaster.actions.QueueAccess;
 import org.apache.slider.server.appmaster.actions.RenewingAction;
 import org.slf4j.Logger;
@@ -36,13 +37,7 @@ import java.util.concurrent.TimeUnit;
 public class ChaosMonkeyService extends AbstractService {
   protected static final Logger log =
       LoggerFactory.getLogger(ChaosMonkeyService.class);
-  public static final int PERCENT_1 = 100;
-  public static final double PERCENT_1D = 100.0;
-  
-  /**
-   * the percentage value as multiplied up
-   */
-  public static final int PERCENT_100 = 100 * PERCENT_1;
+
   private final MetricRegistry metrics;
   private final QueueAccess queues;
   private final Random random = new Random();
@@ -65,7 +60,7 @@ public class ChaosMonkeyService extends AbstractService {
   public synchronized void addTarget(String name,
       ChaosTarget target, long probability) {
     if (probability > 0) {
-      log.info("Adding {} with probability {}", name, probability / PERCENT_1);
+      log.info("Adding {} with probability {}", name, probability / InternalKeys.PROBABILITY_PERCENT_1);
       chaosEntries.add(new ChaosEntry(name, target, probability, metrics));
     } else {
       log.debug("Action {} not enabled", name);
@@ -85,11 +80,25 @@ public class ChaosMonkeyService extends AbstractService {
    */
   public void play() {
     for (ChaosEntry chaosEntry : chaosEntries) {
-      long p = random.nextInt(PERCENT_100);
+      long p = randomPercentage();
       chaosEntry.maybeInvokeChaos(p);
     }
   }
 
+  public int randomPercentage() {
+    return random.nextInt(InternalKeys.PROBABILITY_PERCENT_100);
+  }
+
+  /**
+   * Check for callers to see if chaos should be triggered; shares the
+   * same random number source as the rest of the monkey entries
+   * @param probability probability 
+   * @return true if the action should happen
+   */
+  public boolean chaosCheck(long probability) {
+    return randomPercentage() < probability; 
+  }
+  
   /**
    * Schedule the monkey
    *

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1cd37f57/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/monkey/TestMockMonkey.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/monkey/TestMockMonkey.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/monkey/TestMockMonkey.groovy
index d31c9f6..e4a42fc 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/monkey/TestMockMonkey.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/monkey/TestMockMonkey.groovy
@@ -21,6 +21,7 @@ package org.apache.slider.server.appmaster.model.monkey
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.slider.api.InternalKeys
 import org.apache.slider.server.appmaster.actions.ActionHalt
 import org.apache.slider.server.appmaster.actions.ActionKillContainer
 import org.apache.slider.server.appmaster.actions.QueueService
@@ -66,7 +67,7 @@ class TestMockMonkey extends BaseMockAppStateTest {
   @Test
   public void testMonkeyPlay() throws Throwable {
     ChaosCounter counter = new ChaosCounter()
-    monkey.addTarget("target", counter, ChaosMonkeyService.PERCENT_100)
+    monkey.addTarget("target", counter, InternalKeys.PROBABILITY_PERCENT_100)
     assert 1 == monkey.targetCount;
     monkey.play()
     assert counter.count == 1
@@ -76,7 +77,7 @@ class TestMockMonkey extends BaseMockAppStateTest {
   public void testMonkeySchedule() throws Throwable {
     ChaosCounter counter = new ChaosCounter()
     assert 0 == monkey.targetCount;
-    monkey.addTarget("target", counter, ChaosMonkeyService.PERCENT_100)
+    monkey.addTarget("target", counter, InternalKeys.PROBABILITY_PERCENT_100)
     assert 1 == monkey.targetCount;
     assert monkey.schedule(0, 1, TimeUnit.SECONDS)
     assert 1 == queues.scheduledActions.size()
@@ -105,8 +106,8 @@ class TestMockMonkey extends BaseMockAppStateTest {
   public void testMonkeyPlaySometimes() throws Throwable {
     ChaosCounter counter = new ChaosCounter()
     ChaosCounter counter2 = new ChaosCounter()
-    monkey.addTarget("target1", counter, ChaosMonkeyService.PERCENT_1 * 50)
-    monkey.addTarget("target2", counter2, ChaosMonkeyService.PERCENT_1 * 25)
+    monkey.addTarget("target1", counter, InternalKeys.PROBABILITY_PERCENT_1 * 50)
+    monkey.addTarget("target2", counter2, InternalKeys.PROBABILITY_PERCENT_1 * 25)
 
     for (int i = 0; i < 100; i++) {
       monkey.play()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1cd37f57/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
index 1c65394..7c479af 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
@@ -457,6 +457,7 @@ abstract class CommandTestBase extends SliderTestUtils {
    * @param cluster
    */
   static void setupCluster(String cluster) {
+    describe "setting up $cluster"
     ensureClusterDestroyed(cluster)
   }
 
@@ -655,7 +656,6 @@ abstract class CommandTestBase extends SliderTestUtils {
         SLIDER_CONFIG.getTrimmed(SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL));
     commands.addAll(extraArgs)
     SliderShell shell = new SliderShell(commands)
-    shell.execute()
     if (0 != shell.execute()) {
       // app has failed.
 
@@ -678,7 +678,7 @@ abstract class CommandTestBase extends SliderTestUtils {
     return shell
   }
 
-  public File createAppReportFile() {
+  public static  File createAppReportFile() {
     File reportFile = File.createTempFile(
         "launch",
         ".json",
@@ -693,7 +693,7 @@ abstract class CommandTestBase extends SliderTestUtils {
    * @param option option to probe and use
    * @return the (possibly extended) list
    */
-  public List<String> maybeAddCommandOption(
+  public static List<String> maybeAddCommandOption(
       List<String> args, List<String> commands, String option) {
     if ( SliderUtils.isSet(option)) {
       args.addAll(commands)
@@ -702,7 +702,7 @@ abstract class CommandTestBase extends SliderTestUtils {
     return args
   }
   
-  public SerializedApplicationReport maybeLoadAppReport(File reportFile) {
+  public static SerializedApplicationReport maybeLoadAppReport(File reportFile) {
     if (reportFile.exists() && reportFile.length()> 0) {
       ApplicationReportSerDeser serDeser = new ApplicationReportSerDeser()
       def report = serDeser.fromFile(reportFile)
@@ -711,7 +711,7 @@ abstract class CommandTestBase extends SliderTestUtils {
     return null;
   }  
   
-  public SerializedApplicationReport maybeLookupFromLaunchReport(File launchReport) {
+  public static SerializedApplicationReport maybeLookupFromLaunchReport(File launchReport) {
     def report = maybeLoadAppReport(launchReport)
     if (report) {
       return lookupApplication(report.applicationId)
@@ -725,7 +725,7 @@ abstract class CommandTestBase extends SliderTestUtils {
    * @param id application ID
    * @return an application report or null
    */
-  public SerializedApplicationReport lookupApplication(String id) {
+  public static SerializedApplicationReport lookupApplication(String id) {
     File reportFile = createAppReportFile();
     try {
       def shell = lookup(id, reportFile)
@@ -787,6 +787,7 @@ abstract class CommandTestBase extends SliderTestUtils {
     }
   }
 
+   
   protected void ensureApplicationIsUp(String application) {
     repeatUntilTrue(this.&isApplicationRunning,
         30,
@@ -816,26 +817,58 @@ abstract class CommandTestBase extends SliderTestUtils {
 
   protected boolean isApplicationRunning(Map<String, String> args) {
     String applicationName = args['application'];
-    return isApplicationInState(YarnApplicationState.RUNNING, applicationName);
+    return isApplicationUp(applicationName);
   }
 
   protected boolean isApplicationUp(String applicationName) {
-    return isApplicationInState(YarnApplicationState.RUNNING, applicationName);
+    return isApplicationInState(
+        applicationName,
+        YarnApplicationState.RUNNING
+    );
   }
 
+  protected void ensureYarnApplicationIsUp(String application) {
+    repeatUntilTrue(this.&isApplicationRunning,
+        30,
+        SLIDER_CONFIG.getInt(KEY_TEST_INSTANCE_LAUNCH_TIME,
+            DEFAULT_INSTANCE_LAUNCH_TIME_SECONDS),
+        [application: application],
+        true,
+        'Application did not start, failing test.') {
+      describe "final state of app that tests say is not up"
+      exists(application, true).dumpOutput()
+    }
+  }
+  
   /**
-   * 
+   * is an application in a desired yarn state 
    * @param yarnState
    * @param applicationName
    * @return
    */
-  public static boolean isApplicationInState(YarnApplicationState yarnState, String applicationName) {
+  public static boolean isApplicationInState(
+      String applicationName,
+      YarnApplicationState yarnState) {
     SliderShell shell = slider(
       [ACTION_EXISTS, applicationName, ARG_STATE, yarnState.toString()])
-
     return shell.ret == 0
   }
+  
+  /**
+   * is a yarn application in a desired yarn state 
+   * @param yarnState
+   * @param applicationName
+   * @return
+   */
+  public static boolean isYarnApplicationInState(
+      String applicationId,
+      YarnApplicationState yarnState) {
+    def sar = lookupApplication(applicationId)
+    assert sar != null;
+    return yarnState.toString() == sar.state
+  }
 
+  
   /**
    * Repeat a probe until it succeeds, if it does not execute a failure
    * closure then raise an exception with the supplied message

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1cd37f57/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/SliderShell.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/SliderShell.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/SliderShell.groovy
index 31830d9..ae40d6a 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/SliderShell.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/SliderShell.groovy
@@ -247,7 +247,6 @@ class SliderShell extends Shell {
   Shell exec(Object... args) {
     Process proc = "$shell".execute()
     script = args.join("\n")
-    LOG.debug("${shell} << __EOT__\n${script}\n__EOT__");
     ByteArrayOutputStream baosErr = new ByteArrayOutputStream(4096);
     ByteArrayOutputStream baosOut = new ByteArrayOutputStream(4096);
     proc.consumeProcessOutput(baosOut, baosErr)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1cd37f57/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AMFailuresIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AMFailuresIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AMFailuresIT.groovy
index 10a3dfe..750a3d4 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AMFailuresIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AMFailuresIT.groovy
@@ -92,8 +92,8 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
     killAMUsingVagrantShell()
 
     // Check that the application is not running (and is in ACCEPTED state)
-    assert isApplicationInState(YarnApplicationState.ACCEPTED,
-        APPLICATION_NAME), 
+    assert isApplicationInState(APPLICATION_NAME, YarnApplicationState.ACCEPTED
+    ), 
       'App should be in ACCEPTED state (since AM got killed)'
     log.info("After AM KILL: application {} is in ACCEPTED state", APPLICATION_NAME)
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1cd37f57/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
index 8b123c3..3783a37 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
@@ -48,7 +48,6 @@ public class AgentClusterLifecycleIT extends AgentCommandTestBase
   @Before
   public void prepareCluster() {
     setupCluster(CLUSTER)
-    describe("Create a 0-role cluster, so testing AM start/stop")
   }
 
   @After
@@ -62,7 +61,7 @@ public class AgentClusterLifecycleIT extends AgentCommandTestBase
     describe "Walk a 0-role cluster through its lifecycle"
 
     // sanity check to verify the config is correct
-    assert clusterFS.uri.scheme!="file"
+    assert clusterFS.uri.scheme != "file"
 
     def clusterpath = buildClusterPath(CLUSTER)
     assert !clusterFS.exists(clusterpath)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1cd37f57/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
index 1a0d2c3..4dcbed0 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
@@ -20,7 +20,7 @@ package org.apache.slider.funtest.lifecycle
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.slider.api.InternalKeys
+import static org.apache.slider.api.InternalKeys.*
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
@@ -41,7 +41,6 @@ public class AgentLaunchFailureIT extends AgentCommandTestBase
 
   static String APP_RESOURCE2 = "../slider-core/src/test/app_packages/test_command_log/resources_no_role.json"
 
-
   @Before
   public void prepareCluster() {
     setupCluster(CLUSTER)
@@ -59,16 +58,15 @@ public class AgentLaunchFailureIT extends AgentCommandTestBase
     // verify no cluster
     assert 0 != exists(CLUSTER).ret
  
-    // create an AM which fails to launch within a second
+    // create an AM which fails to launch
     File launchReportFile = createAppReportFile();
     SliderShell shell = createTemplatedSliderApplication(CLUSTER,
         APP_TEMPLATE,
         APP_RESOURCE2,
         [
-            ARG_OPTION, InternalKeys.CHAOS_MONKEY_ENABLED, "true",
-            ARG_OPTION, InternalKeys.CHAOS_MONKEY_DELAY_SECONDS, "1",
-            ARG_OPTION, InternalKeys.CHAOS_MONKEY_INTERVAL_SECONDS, "60",
-            ARG_OPTION, InternalKeys.CHAOS_MONKEY_PROBABILITY_AM_FAILURE, "100",
+            ARG_OPTION, CHAOS_MONKEY_ENABLED, "true",
+            ARG_OPTION, CHAOS_MONKEY_PROBABILITY_AM_LAUNCH_FAILURE, 
+             Integer.toString(PROBABILITY_PERCENT_100),
         ],
         launchReportFile)
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1cd37f57/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentDemo.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentDemo.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentDemo.groovy
new file mode 100644
index 0000000..eb825fc
--- /dev/null
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentDemo.groovy
@@ -0,0 +1,160 @@
+/*
+ * 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.slider.funtest.lifecycle
+
+import groovy.transform.CompileStatic
+import groovy.util.logging.Slf4j
+import org.apache.slider.common.SliderExitCodes
+import org.apache.slider.common.params.Arguments
+import org.apache.slider.common.params.SliderActions
+import org.apache.slider.funtest.framework.AgentCommandTestBase
+import org.apache.slider.funtest.framework.FuntestProperties
+import org.apache.slider.funtest.framework.SliderShell
+import org.junit.After
+import org.junit.Before
+import org.junit.Test
+
+@CompileStatic
+@Slf4j
+public class AppsThroughAgentDemo extends AgentCommandTestBase
+implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
+
+  private static String COMMAND_LOGGER = "COMMAND_LOGGER"
+  private static String APPLICATION_NAME = "agent-demo"
+
+
+  @Before
+  public void prepareCluster() {
+    setupCluster(APPLICATION_NAME)
+  }
+
+  @Test
+  public void testCreateFlex() throws Throwable {
+    assumeAgentTestsEnabled()
+
+    cleanup(APPLICATION_NAME)
+    SliderShell shell = createTemplatedSliderApplication(APPLICATION_NAME,
+        APP_TEMPLATE,
+        APP_RESOURCE)
+
+    logShell(shell)
+
+    ensureApplicationIsUp(APPLICATION_NAME)
+
+    //flex
+    slider(EXIT_SUCCESS,
+        [
+            ACTION_FLEX,
+            APPLICATION_NAME,
+            ARG_COMPONENT,
+            COMMAND_LOGGER,
+            "2"])
+
+    // sleep till the new instance starts
+    sleep(1000 * 10)
+
+    shell = slider(EXIT_SUCCESS,
+        [
+            ACTION_STATUS,
+            APPLICATION_NAME])
+
+    expectContainersLive(APPLICATION_NAME, COMMAND_LOGGER, 2)
+
+    String amWebUrl = getInfoAmWebUrl(APPLICATION_NAME)
+    log.info("Dumping data from AM Web URL");
+    log.info(amWebUrl.toURL().text);
+
+    ensureRegistryCallSucceeds(APPLICATION_NAME)
+
+    // get log folders
+    shell = slider(EXIT_SUCCESS,
+        [
+            ACTION_REGISTRY,
+            ARG_NAME,
+            APPLICATION_NAME,
+            ARG_LISTEXP])
+    if(!containsString(shell, "container_log_dirs") ||
+       !containsString(shell, "container_work_dirs")) {
+      logShell(shell)
+      assert fail("Should list default exports container_log_dirs or container_work_dirs")
+    }
+
+    // get log folders
+    shell = slider(EXIT_SUCCESS,
+        [
+            ACTION_REGISTRY,
+            ARG_NAME,
+            APPLICATION_NAME,
+            ARG_GETEXP,
+            "container_log_dirs"])
+    if(!containsString(shell, "\"tag\" : \"COMMAND_LOGGER\"", 2)
+    || !containsString(shell, "\"level\" : \"component\"", 2)) {
+      logShell(shell)
+      assert fail("Should list 2 entries for log folders")
+    }
+
+    // get log folders
+    shell = slider(EXIT_SUCCESS,
+        [
+            ACTION_REGISTRY,
+            ARG_NAME,
+            APPLICATION_NAME,
+            ARG_GETEXP,
+            "container_work_dirs"])
+    if(!containsString(shell, "\"tag\" : \"COMMAND_LOGGER\"", 2)
+    || !containsString(shell, "\"level\" : \"component\"", 2)) {
+      logShell(shell)
+      assert fail("Should list 2 entries for work folder")
+    }
+
+    // get cl-site config
+    shell = slider(
+        [
+            ACTION_REGISTRY,
+            ARG_NAME,
+            APPLICATION_NAME,
+            ARG_GETCONF,
+            "cl-site",
+            ARG_FORMAT,
+            "json"])
+
+    for (int i = 0; i < 10; i++) {
+      if (shell.getRet() != EXIT_SUCCESS) {
+        println "Waiting for the cl-site to show up"
+        sleep(1000 * 10)
+        shell = slider(
+            [
+                ACTION_REGISTRY,
+                ARG_NAME,
+                APPLICATION_NAME,
+                ARG_GETCONF,
+                "cl-site",
+                ARG_FORMAT,
+                "json"])
+      }
+    }
+    assert shell.getRet() == EXIT_SUCCESS, "cl-site should be retrieved"
+    if (!containsString(shell, "\"pattern.for.test.to.verify\" : \"verify this pattern\"", 1)) {
+      logShell(shell)
+      assert fail("Should have exported cl-site")
+    }
+
+    assert isApplicationUp(APPLICATION_NAME), 'App is not running.'
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1cd37f57/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentIT.groovy
index fd8330b..488173b 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentIT.groovy
@@ -27,6 +27,7 @@ import org.apache.slider.funtest.framework.AgentCommandTestBase
 import org.apache.slider.funtest.framework.FuntestProperties
 import org.apache.slider.funtest.framework.SliderShell
 import org.junit.After
+import org.junit.Before
 import org.junit.Test
 
 @CompileStatic
@@ -35,8 +36,13 @@ public class AppsThroughAgentIT extends AgentCommandTestBase
 implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
 
   private static String COMMAND_LOGGER = "COMMAND_LOGGER"
-  private static String APPLICATION_NAME = "happy-path-with-flex"
+  private static String APPLICATION_NAME = "apps-through-agent"
 
+  @Before
+  public void prepareCluster() {
+    setupCluster(APPLICATION_NAME)
+  }
+  
   @After
   public void destroyCluster() {
     cleanup(APPLICATION_NAME)


[07/13] git commit: SLIDER-570 handling of launch failures

Posted by st...@apache.org.
SLIDER-570 handling of launch failures


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/73462659
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/73462659
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/73462659

Branch: refs/heads/develop
Commit: 734626596c0af041c53637ac8053eb50d0a8d169
Parents: 517042f
Author: Steve Loughran <st...@apache.org>
Authored: Wed Oct 29 20:49:39 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Oct 31 11:07:49 2014 +0000

----------------------------------------------------------------------
 .../funtest/framework/CommandTestBase.groovy    | 95 ++++++++++++++++++--
 .../slider/funtest/framework/SliderShell.groovy |  4 +-
 .../funtest/lifecycle/AgentFailuresIT.groovy    |  6 +-
 .../lifecycle/AgentLaunchFailureIT.groovy       | 95 ++++++++++++++++++++
 .../funtest/lifecycle/AgentRegistryIT.groovy    |  6 +-
 5 files changed, 189 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/73462659/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
index 7b50c60..7928642 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
@@ -30,12 +30,15 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.slider.api.StatusKeys
 import org.apache.slider.common.tools.ConfigHelper
+import org.apache.slider.core.exceptions.SliderException
+import org.apache.slider.core.launch.SerializedApplicationReport
 import org.apache.slider.core.main.ServiceLauncher
 import org.apache.slider.common.SliderKeys
 import org.apache.slider.common.SliderXmlConfKeys
 import org.apache.slider.api.ClusterDescription
 import org.apache.slider.common.tools.SliderUtils
 import org.apache.slider.client.SliderClient
+import org.apache.slider.core.persist.ApplicationReportSerDeser
 import org.apache.slider.test.SliderTestUtils
 import org.junit.Before
 import org.junit.BeforeClass
@@ -369,6 +372,20 @@ abstract class CommandTestBase extends SliderTestUtils {
     slider(cmd)
   }
 
+  static SliderShell lookup(int result, String id, File out) {
+    assert id
+    def commands = [ACTION_LOOKUP, ARG_ID, id]
+    if (out) commands += [ARG_OUTPUT, out.absolutePath]
+    slider(result, commands)
+  }
+  
+  static SliderShell lookup(String id, File out) {
+    assert id
+    def commands = [ACTION_LOOKUP, ARG_ID, id]
+    if (out) commands += [ARG_OUTPUT, out.absolutePath]
+    slider(commands)
+  }
+
   static SliderShell list(int result, Collection<String> commands =[]) {
     slider(result, [ACTION_LIST] + commands )
   }
@@ -608,11 +625,18 @@ abstract class CommandTestBase extends SliderTestUtils {
       String name,
       String appTemplate,
       String resourceTemplate,
-      List<String> extraArgs=[]) {
+      List<String> extraArgs = [],
+      File launchReport = null) {
+
+    if (!launchReport) {
+      launchReport = createAppReportFile()
+    }
+    
     List<String> commands = [
         ACTION_CREATE, name,
         ARG_TEMPLATE, appTemplate,
         ARG_RESOURCES, resourceTemplate,
+        ARG_OUTPUT, launchReport.absolutePath,
         ARG_WAIT, Integer.toString(THAW_WAIT_TIME)
     ]
 
@@ -633,20 +657,35 @@ abstract class CommandTestBase extends SliderTestUtils {
     shell.execute()
     if (!shell.execute()) {
       // app has failed.
-      
+
       // grab the app report of the last known instance of this app
       // which may not be there if it was a config failure; may be out of date
       // from a previous run
-      log.error("Launch failed with exit code ${shell.ret}.\nLast instance of $name:")
-      slider([ACTION_LIST, name, ARG_VERBOSE]).dumpOutput()
-      
-      // trigger the assertion failure
-      shell.assertExitCode(EXIT_SUCCESS)
+      log.error(
+          "Launch failed with exit code ${shell.ret}")
+      shell.dumpOutput()
+
+      // now grab that app report if it is there
+      def appReport = maybeLookupFromLaunchReport(launchReport)
+      String extraText = ""
+      if (appReport) {
+        log.error("Application report:\n$appReport")
+        extraText = appReport.diagnostics
+      }
+
+      fail("Application Launch Failure, exit code  ${shell.ret}\n${extraText}")
     }
-    
     return shell
   }
 
+  public File createAppReportFile() {
+    File reportFile = File.createTempFile(
+        "launch",
+        ".json",
+        new File("target"))
+    return reportFile
+  }
+
   /**
    * If the option is not null/empty, add the command and the option
    * @param args arg list being built up
@@ -662,7 +701,47 @@ abstract class CommandTestBase extends SliderTestUtils {
     }
     return args
   }
+  
+  public SerializedApplicationReport maybeLoadAppReport(File reportFile) {
+    if (reportFile.exists() && reportFile.length()> 0) {
+      ApplicationReportSerDeser serDeser = new ApplicationReportSerDeser()
+      def report = serDeser.fromFile(reportFile)
+      return report
+    }    
+    return null;
+  }  
+  
+  public SerializedApplicationReport maybeLookupFromLaunchReport(File launchReport) {
+    def report = maybeLoadAppReport(launchReport)
+    if (report) {
+      return lookupApplication(report.applicationId)
+    } else {
+      return null
+    }
+  }
+
+  /**
+   * Lookup an application, return null if loading failed
+   * @param id application ID
+   * @return an application report or null
+   */
+  public SerializedApplicationReport lookupApplication(String id) {
+    File reportFile = createAppReportFile();
+    try {
+      def shell = lookup(id, reportFile)
+      if (shell.ret) {
+        return maybeLoadAppReport(reportFile)
+      } else {
+        log.warn("Lookup operation failed:\n" + shell.dumpOutput())
+        return null
+      }
+    } finally {
+      reportFile.delete()
+      
+    }
+  }
 
+  
   public Path buildClusterPath(String clustername) {
     return new Path(
         clusterFS.homeDirectory,

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/73462659/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/SliderShell.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/SliderShell.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/SliderShell.groovy
index 43ac477..31830d9 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/SliderShell.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/SliderShell.groovy
@@ -223,11 +223,11 @@ class SliderShell extends Shell {
    * if not the output is printed and an assertion is raised
    * @param errorCode expected error code
    */
-  public void assertExitCode(int errorCode) {
+  public void assertExitCode(int errorCode, String extra="") {
     if (this.ret != errorCode) {
       dumpOutput()
       throw new SliderException(ret,
-          "Expected exit code of command ${command} : ${errorCode} - actual=${ret}")
+          "Expected exit code of command ${command} : ${errorCode} - actual=${ret} $extra")
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/73462659/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
index a4eb1a2..3847e3f 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
@@ -50,9 +50,9 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
     }
 
     cleanup(APPLICATION_NAME)
-    def shell = createTemplatedSliderApplication( APPLICATION_NAME,
-            APP_TEMPLATE2,
-            APP_RESOURCE)
+    def shell = createTemplatedSliderApplication(APPLICATION_NAME,
+        APP_TEMPLATE2,
+        APP_RESOURCE)
 
     logShell(shell)
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/73462659/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
new file mode 100644
index 0000000..ce1e0f1
--- /dev/null
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
@@ -0,0 +1,95 @@
+/*
+ * 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.slider.funtest.lifecycle
+
+import groovy.transform.CompileStatic
+import groovy.util.logging.Slf4j
+import org.apache.hadoop.registry.client.binding.RegistryUtils
+import org.apache.hadoop.registry.client.types.Endpoint
+import org.apache.hadoop.registry.client.types.ServiceRecord
+import org.apache.slider.api.InternalKeys
+import org.apache.slider.common.SliderExitCodes
+import org.apache.slider.common.SliderKeys
+import org.apache.slider.common.params.Arguments
+import org.apache.slider.common.params.SliderActions
+import org.apache.slider.funtest.framework.AgentCommandTestBase
+import org.apache.slider.funtest.framework.FuntestProperties
+import org.apache.slider.funtest.framework.SliderShell
+import org.junit.After
+import org.junit.Before
+import org.junit.Test
+
+import static org.apache.slider.core.registry.info.CustomRegistryConstants.*
+
+@CompileStatic
+@Slf4j
+public class AgentLaunchFailureIT extends AgentCommandTestBase
+    implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
+
+
+  static String CLUSTER = "test-agent-launchfail"
+
+  static String APP_RESOURCE2 = "../slider-core/src/test/app_packages/test_command_log/resources_no_role.json"
+
+
+  @Before
+  public void prepareCluster() {
+    setupCluster(CLUSTER)
+ }
+
+  @After
+  public void destroyCluster() {
+    cleanup(CLUSTER)
+  }
+
+  @Test
+  public void testAgentLaunchFailure() throws Throwable {
+    describe("Create a failing cluster and validate failure logic")
+
+    // create an AM which fails to launch within a second
+    File launchReportFile = createAppReportFile();
+    SliderShell shell = createTemplatedSliderApplication(CLUSTER,
+        APP_TEMPLATE,
+        APP_RESOURCE2,
+        [
+            ARG_INTERNAL, InternalKeys.CHAOS_MONKEY_ENABLED, "true",
+            ARG_INTERNAL, InternalKeys.CHAOS_MONKEY_INTERVAL_SECONDS, "1",
+            ARG_INTERNAL, InternalKeys.CHAOS_MONKEY_PROBABILITY_AM_FAILURE, "100",
+        ],
+        launchReportFile)
+
+    maybeLookupFromLaunchReport(launchReportFile)
+    ensureApplicationIsUp(CLUSTER)
+
+
+    //stop
+    freeze(0, CLUSTER,
+        [
+            ARG_FORCE,
+            ARG_WAIT, Integer.toString(FREEZE_WAIT_TIME),
+            ARG_MESSAGE, "final-shutdown"
+        ])
+
+    destroy(0, CLUSTER)
+
+    //cluster now missing
+    exists(EXIT_UNKNOWN_INSTANCE, CLUSTER)
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/73462659/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
index 50da8ae..16e65fa 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
@@ -49,9 +49,7 @@ public class AgentRegistryIT extends AgentCommandTestBase
   @Before
   public void prepareCluster() {
     setupCluster(CLUSTER)
-
-
-  }
+ }
 
   @After
   public void destroyCluster() {
@@ -59,7 +57,7 @@ public class AgentRegistryIT extends AgentCommandTestBase
   }
 
   @Test
-  public void testAgentClusterLifecycle() throws Throwable {
+  public void testAgentRegistry() throws Throwable {
     describe("Create a 0-role cluster and make registry queries against it")
 
     // sanity check to verify the config is correct


[11/13] git commit: SLIDER-570: tuning of tests, agent failure tests working (i.e. regression fixed)

Posted by st...@apache.org.
SLIDER-570: tuning of tests, agent failure tests working (i.e. regression fixed)


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/e1023ead
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/e1023ead
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/e1023ead

Branch: refs/heads/develop
Commit: e1023eadf896866ce6c63f06e47bf835adc8576c
Parents: 6e171c7
Author: Steve Loughran <st...@apache.org>
Authored: Fri Oct 31 13:25:26 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Oct 31 13:25:26 2014 +0000

----------------------------------------------------------------------
 .../framework/AgentCommandTestBase.groovy       | 10 ++++
 .../funtest/framework/CommandTestBase.groovy    | 54 ++++++++++++--------
 .../funtest/lifecycle/AMFailuresIT.groovy       |  3 +-
 .../lifecycle/AgentClusterLifecycleIT.groovy    |  1 +
 .../funtest/lifecycle/AgentFailures2IT.groovy   | 12 ++---
 .../funtest/lifecycle/AgentFailuresIT.groovy    |  8 +--
 .../lifecycle/AgentLaunchFailureIT.groovy       |  3 +-
 .../funtest/lifecycle/AgentRegistryIT.groovy    |  2 -
 .../lifecycle/AppsThroughAgentDemo.groovy       | 27 ++++++----
 .../funtest/lifecycle/AppsThroughAgentIT.groovy | 16 +++---
 .../AppsThroughAgentQueueAndLabelsIT.groovy     |  6 ++-
 11 files changed, 84 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e1023ead/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentCommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentCommandTestBase.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentCommandTestBase.groovy
index 6c99ab6..6e05056 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentCommandTestBase.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentCommandTestBase.groovy
@@ -21,6 +21,7 @@ package org.apache.slider.funtest.framework
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
@@ -167,4 +168,13 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
       assert fail("Old cluster either should not exist or should get destroyed; destroy exit code = ${shell.ret}")
     }
   }
+
+  /**
+   * Assert that the application is running (i.e in state
+   * {@link YarnApplicationState#RUNNING})
+   * @param appId application ID
+   */
+  def assertAppRunning(String appId) {
+    assertInYarnState(appId, YarnApplicationState.RUNNING)
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e1023ead/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
index fc74129..7ea84df 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
@@ -94,8 +94,7 @@ abstract class CommandTestBase extends SliderTestUtils {
   public static final int CONTAINER_LAUNCH_TIMEOUT = 90000
   public static final int PROBE_SLEEP_TIME = 4000
   public static final int REGISTRY_STARTUP_TIMEOUT = 60000
-  public static
-  final String E_LAUNCH_FAIL = 'Application did not start'
+  public static final String E_LAUNCH_FAIL = 'Application did not start'
 
   /*
   Static initializer for test configurations. If this code throws exceptions
@@ -380,14 +379,18 @@ abstract class CommandTestBase extends SliderTestUtils {
   static SliderShell lookup(int result, String id, File out) {
     assert id
     def commands = [ACTION_LOOKUP, ARG_ID, id]
-    if (out) commands += [ARG_OUTPUT, out.absolutePath]
+    if (out) {
+      commands += [ARG_OUTPUT, out.absolutePath]
+    }
     slider(result, commands)
   }
   
   static SliderShell lookup(String id, File out) {
     assert id
     def commands = [ACTION_LOOKUP, ARG_ID, id]
-    if (out) commands += [ARG_OUTPUT, out.absolutePath]
+    if (out) {
+      commands += [ARG_OUTPUT, out.absolutePath]
+    }
     slider(commands)
   }
 
@@ -707,7 +710,7 @@ abstract class CommandTestBase extends SliderTestUtils {
     }
     return args
   }
-  
+
   public static SerializedApplicationReport maybeLoadAppReport(File reportFile) {
     if (reportFile.exists() && reportFile.length()> 0) {
       ApplicationReportSerDeser serDeser = new ApplicationReportSerDeser()
@@ -776,19 +779,24 @@ abstract class CommandTestBase extends SliderTestUtils {
         ARG_MESSAGE, "suicide"
     ])
 
-
     sleep(5000)
     ensureApplicationIsUp(cluster)
-    
-/*
-    def sleeptime = SLIDER_CONFIG.getInt(KEY_AM_RESTART_SLEEP_TIME,
-        DEFAULT_AM_RESTART_SLEEP_TIME)
-    sleep(sleeptime)
-*/
-    ClusterDescription status
+    return sliderClient.clusterDescription
+  }
+  public ClusterDescription killAmAndWaitForRestart(
+      SliderClient sliderClient, String cluster, String appId) {
+
+    assert cluster
+    slider(0, [
+        ACTION_AM_SUICIDE, cluster,
+        ARG_EXITCODE, "1",
+        ARG_WAIT, "1000",
+        ARG_MESSAGE, "suicide"
+    ])
 
-    status = sliderClient.clusterDescription
-    return status
+    sleep(5000)
+    ensureYarnApplicationIsUp(appId)
+    return sliderClient.clusterDescription
   }
 
   protected void ensureRegistryCallSucceeds(String application) {
@@ -844,7 +852,6 @@ abstract class CommandTestBase extends SliderTestUtils {
     );
   }
 
-  
   /**
    * is an application in a desired yarn state 
    * @param yarnState
@@ -1057,25 +1064,30 @@ abstract class CommandTestBase extends SliderTestUtils {
     int expectedCount = args['limit'].toInteger();
 
     int requestedCount = queryRequestedCount(application, role)
-    log.debug("requested count = $requestedCount; expected=$expectedCount")
+    log.debug("requested $role count = $requestedCount; expected=$expectedCount")
     return Outcome.fromBool(requestedCount >= expectedCount)
   }
 
-  void expectContainerRequestedCountReached(String application, String role, int limit) {
+  void expectContainerRequestedCountReached(String application, String role, int limit,
+      int container_launch_timeout) {
 
     repeatUntilSuccess(
         this.&hasRequestedContainerCountReached,
-        CONTAINER_LAUNCH_TIMEOUT,
+        container_launch_timeout,
         PROBE_SLEEP_TIME,
         [limit      : Integer.toString(limit),
          role       : role,
          application: application],
         true,
         "countainer count not reached") {
-      describe "container count not reached"
+      int requestedCount = queryRequestedCount(application, role)
+
+      def message = "expected count of $role = $limit not reached: $requestedCount" +
+                    " after $container_launch_timeout mS"
+      describe message
       ClusterDescription cd = execStatus(application);
       log.info("Parsed status \n$cd")
-      status(application).dumpOutput()
+      fail(message)
     };
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e1023ead/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AMFailuresIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AMFailuresIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AMFailuresIT.groovy
index 2e28c84..7cc01b8 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AMFailuresIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AMFailuresIT.groovy
@@ -70,7 +70,8 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
     logShell(shell)
 
     def appId = ensureYarnApplicationIsUp(launchReportFile)
-    expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 1)
+    expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 1,
+        CONTAINER_LAUNCH_TIMEOUT)
     
     // Wait for 20 secs for AM and agent to both reach STARTED state
     sleep(1000 * 20)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e1023ead/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
index 3e5cec7..cb137ce 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
@@ -183,6 +183,7 @@ public class AgentClusterLifecycleIT extends AgentCommandTestBase
 
       //start with a restart count set to enable restart
       describe "the kill/restart phase may fail if yarn.resourcemanager.am.max-attempts is too low"
+      
       thaw(CLUSTER,
           [
               ARG_WAIT, Integer.toString(THAW_WAIT_TIME),

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e1023ead/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailures2IT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailures2IT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailures2IT.groovy
index 6c6b52b..bfae6ec 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailures2IT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailures2IT.groovy
@@ -20,7 +20,6 @@ package org.apache.slider.funtest.lifecycle
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
@@ -60,13 +59,12 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
     logShell(shell)
 
     def appId = ensureYarnApplicationIsUp(launchReportFile)
-    expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 3)
+    expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 2,
+        CONTAINER_LAUNCH_TIMEOUT * 2)
     sleep(1000 * 20)
-    def cd = execStatus(APPLICATION_NAME)
-    assert cd.statistics[COMMAND_LOGGER]["containers.requested"] >= 3
-    assertInYarnState(appId, YarnApplicationState.RUNNING)
+    expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 3,
+        CONTAINER_LAUNCH_TIMEOUT * 2)
+    assertAppRunning(appId)
   }
 
-
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e1023ead/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
index 0410881..efd6194 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
@@ -63,13 +63,13 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
     logShell(shell)
 
     def appId = ensureYarnApplicationIsUp(launchReportFile)
-
-    expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 2)
+    expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 2,
+        CONTAINER_LAUNCH_TIMEOUT)
     sleep(1000 * 20)
-    assertInYarnState(appId, YarnApplicationState.RUNNING)
+    assertAppRunning(appId)
     def cd = expectContainersLive(APPLICATION_NAME, COMMAND_LOGGER, 1)
     assert cd.statistics[COMMAND_LOGGER]["containers.requested"] >= 2
-    assertInYarnState(appId, YarnApplicationState.RUNNING)
+    assertAppRunning(appId)
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e1023ead/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
index 5087c06..791123c 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
@@ -28,7 +28,6 @@ import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
 import org.apache.slider.funtest.framework.AgentCommandTestBase
 import org.apache.slider.funtest.framework.FuntestProperties
-import org.apache.slider.funtest.framework.SliderShell
 import org.junit.After
 import org.junit.Before
 import org.junit.Test
@@ -62,7 +61,7 @@ public class AgentLaunchFailureIT extends AgentCommandTestBase
  
     // create an AM which fails to launch
     File launchReportFile = createAppReportFile();
-    SliderShell shell = createTemplatedSliderApplication(CLUSTER,
+    createTemplatedSliderApplication(CLUSTER,
         APP_TEMPLATE,
         APP_RESOURCE2,
         [

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e1023ead/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
index b5ee23d..5b8b3cd 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
@@ -132,7 +132,5 @@ public class AgentRegistryIT extends AgentCommandTestBase
 
     //cluster now missing
     exists(EXIT_UNKNOWN_INSTANCE, CLUSTER)
-
-
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e1023ead/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentDemo.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentDemo.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentDemo.groovy
index eb825fc..5be7211 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentDemo.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentDemo.groovy
@@ -49,13 +49,17 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
     assumeAgentTestsEnabled()
 
     cleanup(APPLICATION_NAME)
+    File launchReportFile = createAppReportFile();
+
     SliderShell shell = createTemplatedSliderApplication(APPLICATION_NAME,
         APP_TEMPLATE,
-        APP_RESOURCE)
+        APP_RESOURCE,
+        [],
+        launchReportFile)
 
     logShell(shell)
 
-    ensureApplicationIsUp(APPLICATION_NAME)
+    def appId = ensureYarnApplicationIsUp(launchReportFile)
 
     //flex
     slider(EXIT_SUCCESS,
@@ -69,9 +73,8 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
     // sleep till the new instance starts
     sleep(1000 * 10)
 
-    shell = slider(EXIT_SUCCESS,
-        [
-            ACTION_STATUS,
+    slider(EXIT_SUCCESS,
+        [ACTION_STATUS,
             APPLICATION_NAME])
 
     expectContainersLive(APPLICATION_NAME, COMMAND_LOGGER, 2)
@@ -88,7 +91,8 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
             ACTION_REGISTRY,
             ARG_NAME,
             APPLICATION_NAME,
-            ARG_LISTEXP])
+            ARG_LISTEXP
+        ])
     if(!containsString(shell, "container_log_dirs") ||
        !containsString(shell, "container_work_dirs")) {
       logShell(shell)
@@ -102,7 +106,8 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
             ARG_NAME,
             APPLICATION_NAME,
             ARG_GETEXP,
-            "container_log_dirs"])
+            "container_log_dirs"
+        ])
     if(!containsString(shell, "\"tag\" : \"COMMAND_LOGGER\"", 2)
     || !containsString(shell, "\"level\" : \"component\"", 2)) {
       logShell(shell)
@@ -116,7 +121,8 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
             ARG_NAME,
             APPLICATION_NAME,
             ARG_GETEXP,
-            "container_work_dirs"])
+            "container_work_dirs"
+        ])
     if(!containsString(shell, "\"tag\" : \"COMMAND_LOGGER\"", 2)
     || !containsString(shell, "\"level\" : \"component\"", 2)) {
       logShell(shell)
@@ -132,7 +138,8 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
             ARG_GETCONF,
             "cl-site",
             ARG_FORMAT,
-            "json"])
+            "json"
+        ])
 
     for (int i = 0; i < 10; i++) {
       if (shell.getRet() != EXIT_SUCCESS) {
@@ -155,6 +162,6 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
       assert fail("Should have exported cl-site")
     }
 
-    assert isApplicationUp(APPLICATION_NAME), 'App is not running.'
+    assertAppRunning(appId)
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e1023ead/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentIT.groovy
index 5a5b964..75807c3 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentIT.groovy
@@ -20,7 +20,6 @@ package org.apache.slider.funtest.lifecycle
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
@@ -76,10 +75,7 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
     // sleep till the new instance starts
     sleep(1000 * 10)
 
-    shell = slider(EXIT_SUCCESS,
-        [
-            ACTION_STATUS,
-            APPLICATION_NAME])
+    status(0, APPLICATION_NAME)
 
     expectContainersLive(APPLICATION_NAME, COMMAND_LOGGER, 2)
 
@@ -145,7 +141,7 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
       if (shell.getRet() != EXIT_SUCCESS) {
         println "Waiting for the cl-site to show up"
         sleep(1000 * 10)
-        shell = slider(
+        shell = slider(0,
             [
                 ACTION_REGISTRY,
                 ARG_NAME,
@@ -156,12 +152,14 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
                 "json"])
       }
     }
-    assert shell.getRet() == EXIT_SUCCESS, "cl-site should be retrieved"
     if (!containsString(shell, "\"pattern.for.test.to.verify\" : \"verify this pattern\"", 1)) {
       logShell(shell)
-      assert fail("Should have exported cl-site")
+      
+      fail("Should have exported cl-site; got " +
+                  "stdout"  +shell.stdErrHistory +
+                  " \nstderr:" + shell.stdErrHistory)
     }
 
-    assertInYarnState(appId,  YarnApplicationState.RUNNING)
+    assertAppRunning(appId)
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e1023ead/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentQueueAndLabelsIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentQueueAndLabelsIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentQueueAndLabelsIT.groovy
index ee418dc..f6a1b1e 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentQueueAndLabelsIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentQueueAndLabelsIT.groovy
@@ -92,7 +92,8 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
 
     def appId = ensureYarnApplicationIsUp(launchReportFile)
 
-    expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 1 )
+    expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 1,
+        CONTAINER_LAUNCH_TIMEOUT)
     expectContainersLive(APPLICATION_NAME, COMMAND_LOGGER, 1)
 
     //flex
@@ -107,7 +108,8 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
 
     // spin till the flexed instance starts
     ensureYarnApplicationIsUp(appId)
-    expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 3)
+    expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 3,
+        CONTAINER_LAUNCH_TIMEOUT)
 
 
     sleep(1000 * 20)


[10/13] git commit: SLIDER-570 migrate tests to app-id based startup cycle

Posted by st...@apache.org.
SLIDER-570 migrate tests to app-id based startup cycle


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/e7df654f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/e7df654f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/e7df654f

Branch: refs/heads/develop
Commit: e7df654f29e414b822a93738321940dc8f164114
Parents: 1cd37f5
Author: Steve Loughran <st...@apache.org>
Authored: Thu Oct 30 22:40:14 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Oct 31 11:07:50 2014 +0000

----------------------------------------------------------------------
 .../apache/slider/core/conf/MapOperations.java  |  14 +-
 .../server/appmaster/SliderAppMaster.java       |   2 +
 .../slider/core/conf/TestConfTreeResolve.groovy |  22 ++-
 .../slider/core/conf/examples/internal.json     |   5 +-
 .../funtest/framework/CommandTestBase.groovy    | 193 ++++++++++++++-----
 .../funtest/lifecycle/AMFailuresIT.groovy       |  16 +-
 .../lifecycle/AgentClusterLifecycleIT.groovy    |   2 +-
 .../funtest/lifecycle/AgentFailures2IT.groovy   |  16 +-
 .../funtest/lifecycle/AgentFailuresIT.groovy    |  16 +-
 .../lifecycle/AgentLaunchFailureIT.groovy       |  34 ++--
 .../funtest/lifecycle/AgentRegistryIT.groovy    |  16 +-
 .../funtest/lifecycle/AppsThroughAgentIT.groovy |  15 +-
 .../AppsThroughAgentQueueAndLabelsIT.groovy     |  22 ++-
 13 files changed, 259 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e7df654f/slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java b/slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java
index de8fc2c..5f7b5f0 100644
--- a/slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java
+++ b/slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java
@@ -38,6 +38,10 @@ import java.util.Set;
 public class MapOperations implements Map<String, String> {
   private static final Logger log =
     LoggerFactory.getLogger(MapOperations.class);
+  public static final String DAYS = ".days";
+  public static final String HOURS = ".hours";
+  public static final String MINUTES = ".minutes";
+  public static final String SECONDS = ".seconds";
 
   /**
    * Global options
@@ -277,7 +281,7 @@ public class MapOperations implements Map<String, String> {
 
   /**
    * Get the time range of a set of keys
-   * @param basekey
+   * @param basekey base key to which suffix gets applied
    * @param defDays
    * @param defHours
    * @param defMins
@@ -290,11 +294,11 @@ public class MapOperations implements Map<String, String> {
       int defMins,
       int defSecs) {
     Preconditions.checkArgument(basekey != null);
-    int days = getOptionInt(basekey + ".days", defDays);
-    int hours = getOptionInt(basekey + ".hours", defHours);
+    int days = getOptionInt(basekey + DAYS, defDays);
+    int hours = getOptionInt(basekey + HOURS, defHours);
 
-    int minutes = getOptionInt(basekey + ".minutes", defMins);
-    int seconds = getOptionInt(basekey + ".seconds", defSecs);
+    int minutes = getOptionInt(basekey + MINUTES, defMins);
+    int seconds = getOptionInt(basekey + SECONDS, defSecs);
     // range check
     Preconditions.checkState(days >= 0 && hours >= 0 && minutes >= 0
                              && seconds >= 0,

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e7df654f/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index 5e2ba11..f2ea00f 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -2199,12 +2199,14 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
         InternalKeys.CHAOS_MONKEY_PROBABILITY_AM_LAUNCH_FAILURE,
         0);
     if (amLaunchFailProbability> 0 && monkey.chaosCheck(amLaunchFailProbability)) {
+      log.info("Chaos Monkey has triggered AM Launch failure");
       // trigger a failure
       ActionStopSlider stop = new ActionStopSlider("stop",
           0, TimeUnit.SECONDS,
           LauncherExitCodes.EXIT_FALSE,
           FinalApplicationStatus.FAILED,
           E_TRIGGERED_LAUNCH_FAILURE);
+      queue(stop);
     }
     
     int amKillProbability = internals.getOptionInt(

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e7df654f/slider-core/src/test/groovy/org/apache/slider/core/conf/TestConfTreeResolve.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/core/conf/TestConfTreeResolve.groovy b/slider-core/src/test/groovy/org/apache/slider/core/conf/TestConfTreeResolve.groovy
index b655be8..156ae71 100644
--- a/slider-core/src/test/groovy/org/apache/slider/core/conf/TestConfTreeResolve.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/core/conf/TestConfTreeResolve.groovy
@@ -19,10 +19,11 @@
 package org.apache.slider.core.conf
 
 import groovy.util.logging.Slf4j
+import static org.apache.slider.api.InternalKeys.*
 import org.junit.Assert
 import org.junit.Test
 
-import static org.apache.slider.core.conf.ExampleConfResources.overridden
+import static org.apache.slider.core.conf.ExampleConfResources.*
 
 /**
  * Test 
@@ -89,4 +90,23 @@ class TestConfTreeResolve extends Assert {
     assert worker["timeout"] == "1000"
 
   }
+
+  @Test
+  public void testTimeIntervalLoading() throws Throwable {
+
+    def orig = ExampleConfResources.loadResource(internal)
+
+    MapOperations internals = new MapOperations(orig.global)
+    def s = internals.getOptionInt(
+        CHAOS_MONKEY_INTERVAL + MapOperations.SECONDS,
+        0)
+    assert s == 60
+    long monkeyInterval = internals.getTimeRange(
+        CHAOS_MONKEY_INTERVAL,
+        DEFAULT_CHAOS_MONKEY_INTERVAL_DAYS,
+        DEFAULT_CHAOS_MONKEY_INTERVAL_HOURS,
+        DEFAULT_CHAOS_MONKEY_INTERVAL_MINUTES,
+        0);
+    assert monkeyInterval == 60;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e7df654f/slider-core/src/test/resources/org/apache/slider/core/conf/examples/internal.json
----------------------------------------------------------------------
diff --git a/slider-core/src/test/resources/org/apache/slider/core/conf/examples/internal.json b/slider-core/src/test/resources/org/apache/slider/core/conf/examples/internal.json
index 8617d1f..4c782fb 100644
--- a/slider-core/src/test/resources/org/apache/slider/core/conf/examples/internal.json
+++ b/slider-core/src/test/resources/org/apache/slider/core/conf/examples/internal.json
@@ -7,7 +7,10 @@
   "global": {
     "application.name": "small_cluster",
     "application.type": "hbase",
-    "application": "hdfs://cluster:8020/apps/hbase/v/1.0.0/application.tar"
+    "application": "hdfs://cluster:8020/apps/hbase/v/1.0.0/application.tar",
+    "internal.chaos.monkey.probability.amlaunchfailure": "10000",
+    "internal.chaos.monkey.interval.seconds": "60",
+    "internal.chaos.monkey.enabled": "true"
   },
   "components": {
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e7df654f/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
index 7c479af..0a7b295 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.slider.api.StatusKeys
 import org.apache.slider.common.tools.ConfigHelper
+import org.apache.slider.common.tools.Duration
 import org.apache.slider.core.launch.SerializedApplicationReport
 import org.apache.slider.core.main.ServiceLauncher
 import org.apache.slider.common.SliderKeys
@@ -90,7 +91,12 @@ abstract class CommandTestBase extends SliderTestUtils {
    * not need to be escaped
    */
   public static final String TILDE
-  
+  public static final int CONTAINER_LAUNCH_TIMEOUT = 90000
+  public static final int PROBE_SLEEP_TIME = 4000
+  public static final int REGISTRY_STARTUP_TIMEOUT = 60000
+  public static
+  final String E_LAUNCH_FAIL = 'Application did not start'
+
   /*
   Static initializer for test configurations. If this code throws exceptions
   (which it may) the class will not be instantiable.
@@ -710,6 +716,16 @@ abstract class CommandTestBase extends SliderTestUtils {
     }    
     return null;
   }  
+   
+  public static SerializedApplicationReport loadAppReport(File reportFile) {
+    if (reportFile.exists() && reportFile.length()> 0) {
+      ApplicationReportSerDeser serDeser = new ApplicationReportSerDeser()
+      def report = serDeser.fromFile(reportFile)
+      return report
+    }  else {
+      throw new FileNotFoundException(reportFile.absolutePath)
+    }  
+  }  
   
   public static SerializedApplicationReport maybeLookupFromLaunchReport(File launchReport) {
     def report = maybeLoadAppReport(launchReport)
@@ -776,9 +792,9 @@ abstract class CommandTestBase extends SliderTestUtils {
   }
 
   protected void ensureRegistryCallSucceeds(String application) {
-    repeatUntilTrue(this.&isRegistryAccessible,
-        10,
-        5 * 1000,
+    repeatUntilSuccess(this.&isRegistryAccessible,
+        REGISTRY_STARTUP_TIMEOUT,
+        PROBE_SLEEP_TIME,
         [application: application],
         true,
         'Application registry is not accessible, failing test.') {
@@ -789,35 +805,36 @@ abstract class CommandTestBase extends SliderTestUtils {
 
    
   protected void ensureApplicationIsUp(String application) {
-    repeatUntilTrue(this.&isApplicationRunning,
-        30,
+    repeatUntilSuccess(this.&isApplicationRunning,
         SLIDER_CONFIG.getInt(KEY_TEST_INSTANCE_LAUNCH_TIME,
-            DEFAULT_INSTANCE_LAUNCH_TIME_SECONDS),
+            DEFAULT_INSTANCE_LAUNCH_TIME_SECONDS) * 1000,
+        PROBE_SLEEP_TIME,
         [application: application],
         true,
-        'Application did not start, failing test.') {
+        E_LAUNCH_FAIL) {
       describe "final state of app that tests say is not up"
       exists(application, true).dumpOutput()
     }
   }
 
-  protected boolean isRegistryAccessible(Map<String, String> args) {
+  protected Outcome isRegistryAccessible(Map<String, String> args) {
     String applicationName = args['application'];
     SliderShell shell = slider(
         [
             ACTION_REGISTRY,
             ARG_NAME,
             applicationName,
-            ARG_LISTEXP])
+            ARG_LISTEXP
+        ])
     if (EXIT_SUCCESS != shell.execute()) {
       logShell(shell)
     }
-    return EXIT_SUCCESS == shell.execute()
+    return Outcome.fromBool(EXIT_SUCCESS == shell.execute())
   }
 
-  protected boolean isApplicationRunning(Map<String, String> args) {
+  protected Outcome isApplicationRunning(Map<String, String> args) {
     String applicationName = args['application'];
-    return isApplicationUp(applicationName);
+    return Outcome.fromBool(isApplicationUp(applicationName))
   }
 
   protected boolean isApplicationUp(String applicationName) {
@@ -827,18 +844,6 @@ abstract class CommandTestBase extends SliderTestUtils {
     );
   }
 
-  protected void ensureYarnApplicationIsUp(String application) {
-    repeatUntilTrue(this.&isApplicationRunning,
-        30,
-        SLIDER_CONFIG.getInt(KEY_TEST_INSTANCE_LAUNCH_TIME,
-            DEFAULT_INSTANCE_LAUNCH_TIME_SECONDS),
-        [application: application],
-        true,
-        'Application did not start, failing test.') {
-      describe "final state of app that tests say is not up"
-      exists(application, true).dumpOutput()
-    }
-  }
   
   /**
    * is an application in a desired yarn state 
@@ -853,27 +858,111 @@ abstract class CommandTestBase extends SliderTestUtils {
       [ACTION_EXISTS, applicationName, ARG_STATE, yarnState.toString()])
     return shell.ret == 0
   }
-  
+
+
+  protected Outcome isYarnApplicationRunning(Map<String, String> args) {
+    String applicationId = args['applicationId'];
+    return isYarnApplicationRunning(applicationId)
+  }
+
   /**
    * is a yarn application in a desired yarn state 
    * @param yarnState
    * @param applicationName
-   * @return
+   * @return an outcome indicating whether the app is at the state, on its way
+   * or has gone past
    */
-  public static boolean isYarnApplicationInState(
-      String applicationId,
-      YarnApplicationState yarnState) {
+  public static Outcome isYarnApplicationRunning(
+      String applicationId) {
+    YarnApplicationState appState = lookupYarnAppState(applicationId)
+    YarnApplicationState yarnState = YarnApplicationState.RUNNING
+    if (yarnState == appState) {
+      return Outcome.Success;
+    }
+    
+    if (appState.ordinal() > yarnState.ordinal()) {
+      // app has passed beyond hope
+      return Outcome.Fail
+    }
+    return Outcome.Retry
+  }
+
+  public static YarnApplicationState lookupYarnAppState(String applicationId) {
     def sar = lookupApplication(applicationId)
     assert sar != null;
-    return yarnState.toString() == sar.state
+    YarnApplicationState appState = YarnApplicationState.valueOf(sar.state)
+    return appState
+  }
+
+  public static void assertInYarnState(String applicationId,
+      YarnApplicationState expectedState) {
+    def applicationReport = lookupApplication(applicationId)
+    assert expectedState.toString() == applicationReport.state 
   }
 
+  /**
+   * Wait for the YARN app to come up. This will fail fast
+   * @param launchReportFile launch time file containing app id
+   * @return the app ID
+   */
+  protected String ensureYarnApplicationIsUp(File launchReportFile) {
+    def id = loadAppReport(launchReportFile).applicationId
+    ensureYarnApplicationIsUp(id)
+    return id;
+  }
+  /**
+   * Wait for the YARN app to come up. This will fail fast
+   * @param applicationId
+   */
+  protected void ensureYarnApplicationIsUp(String applicationId) {
+    repeatUntilSuccess(this.&isYarnApplicationRunning,
+        SLIDER_CONFIG.getInt(KEY_TEST_INSTANCE_LAUNCH_TIME,
+            DEFAULT_INSTANCE_LAUNCH_TIME_SECONDS),
+        PROBE_SLEEP_TIME,
+        [applicationId: applicationId],
+        true,
+        E_LAUNCH_FAIL) {
+      describe "final state of app that tests say is not up"
+      def sar = lookupApplication(applicationId)
+
+      def message = E_LAUNCH_FAIL + "\n$sar"
+      log.error(message)
+      fail(message)
+    }
+  }
+
+  /**
+   * Outcome for probes
+   */
+  static class Outcome {
+
+    public final String name;
+
+    private Outcome(String name) {
+      this.name = name
+    }
+
+    static Outcome Success = new Outcome("Success")
+    static Outcome Retry = new Outcome("Retry")
+    static Outcome Fail = new Outcome("Fail")
+
+
+    /**
+     * build from a bool, where false is mapped to retry
+     * @param b boolean
+     * @return an outcome
+     */
+    static Outcome fromBool(boolean b) {
+      return b? Success: Retry;
+    }
+
+  }
   
   /**
    * Repeat a probe until it succeeds, if it does not execute a failure
    * closure then raise an exception with the supplied message
    * @param probe probe
-   * @param maxAttempts max number of attempts
+   * @param timeout time in millis before giving up
    * @param sleepDur sleep between failing attempts
    * @param args map of arguments to the probe
    * @param failIfUnsuccessful if the probe fails after all the attempts
@@ -881,23 +970,35 @@ abstract class CommandTestBase extends SliderTestUtils {
    * @param failureMessage message to include in exception raised
    * @param failureHandler closure to invoke prior to the failure being raised
    */
-  protected void repeatUntilTrue(Closure probe,
-      int maxAttempts, int sleepDur, Map args,
-      boolean failIfUnsuccessful = false,
+  protected void repeatUntilSuccess(Closure probe,
+      int timeout, int sleepDur,
+      Map args,
+      boolean failIfUnsuccessful,
       String failureMessage,
       Closure failureHandler) {
     int attemptCount = 0
     boolean succeeded = false;
-    while (attemptCount < maxAttempts) {
-      if (probe(args)) {
-        // finished
+    boolean completed = false;
+    Duration duration = new Duration(timeout)
+    duration.start();
+    while (!completed) {
+      Outcome outcome = (Outcome) probe(args)
+      if (outcome.equals(Outcome.Success)) {
+        // success
         log.debug("Success after $attemptCount attempt(s)")
         succeeded = true;
-        break
-      };
-      attemptCount++;
-
-      sleep(sleepDur)
+        completed = true;
+      } else if (outcome.equals(Outcome.Retry)) {
+        // failed but retry possible
+        attemptCount++;
+        completed = duration.limitExceeded
+        if (!completed) {
+          sleep(sleepDur)
+        }
+      } else if (outcome.equals(Outcome.Fail)) {
+        // fast fail
+          completed = true;
+      }
     }
     
     if (failIfUnsuccessful & !succeeded) {
@@ -962,10 +1063,10 @@ abstract class CommandTestBase extends SliderTestUtils {
 
   void expectContainerRequestedCountReached(String application, String role, int limit) {
 
-    repeatUntilTrue(
+    repeatUntilSuccess(
         this.&hasRequestedContainerCountReached,
-        90,
-        1000,
+        CONTAINER_LAUNCH_TIMEOUT,
+        PROBE_SLEEP_TIME,
         [limit      : Integer.toString(limit),
          role       : role,
          application: application],

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e7df654f/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AMFailuresIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AMFailuresIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AMFailuresIT.groovy
index 750a3d4..2e28c84 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AMFailuresIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AMFailuresIT.groovy
@@ -61,12 +61,15 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
   @Test
   public void testAMKilledWithStateAMStartedAgentsStarted() throws Throwable {
     cleanup(APPLICATION_NAME)
+    File launchReportFile = createAppReportFile();
+
     SliderShell shell = createTemplatedSliderApplication(
-        APPLICATION_NAME, APP_TEMPLATE, APP_RESOURCE
-    )
+        APPLICATION_NAME, APP_TEMPLATE, APP_RESOURCE,
+        [],
+        launchReportFile)
     logShell(shell)
 
-    ensureApplicationIsUp(APPLICATION_NAME)
+    def appId = ensureYarnApplicationIsUp(launchReportFile)
     expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 1)
     
     // Wait for 20 secs for AM and agent to both reach STARTED state
@@ -92,13 +95,12 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
     killAMUsingVagrantShell()
 
     // Check that the application is not running (and is in ACCEPTED state)
-    assert isApplicationInState(APPLICATION_NAME, YarnApplicationState.ACCEPTED
-    ), 
+    assert lookupYarnAppState(appId) == YarnApplicationState.ACCEPTED ,
       'App should be in ACCEPTED state (since AM got killed)'
     log.info("After AM KILL: application {} is in ACCEPTED state", APPLICATION_NAME)
 
     // Wait until AM comes back up and verify container count again
-    ensureApplicationIsUp(APPLICATION_NAME)
+    ensureYarnApplicationIsUp(appId)
 
     // There should be exactly 1 live logger container
     def cd2 = expectContainersLive(APPLICATION_NAME, COMMAND_LOGGER, 1)
@@ -107,7 +109,7 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
     def loggerStats2 = cd2.statistics[COMMAND_LOGGER]
     assert origRequested == loggerStats2["containers.requested"],
         'No new agent containers should be requested'
-    assert isApplicationUp(APPLICATION_NAME), 'App is not running.'
+    assert lookupYarnAppState(appId) == YarnApplicationState.RUNNING 
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e7df654f/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
index 3783a37..3e5cec7 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentClusterLifecycleIT.groovy
@@ -78,7 +78,7 @@ public class AgentClusterLifecycleIT extends AgentCommandTestBase
     def launchReport = maybeLoadAppReport(launchReportFile)
     assert launchReport;
 
-    ensureApplicationIsUp(CLUSTER)
+    def appId = ensureYarnApplicationIsUp(launchReportFile)
 
     //at this point the cluster should exist.
     assertPathExists(clusterFS, "Cluster parent directory does not exist", clusterpath.parent)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e7df654f/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailures2IT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailures2IT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailures2IT.groovy
index 9b35fa4..6c6b52b 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailures2IT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailures2IT.groovy
@@ -20,6 +20,7 @@ package org.apache.slider.funtest.lifecycle
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
@@ -50,21 +51,20 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
     assumeAgentTestsEnabled()
     
     cleanup(APPLICATION_NAME)
-    SliderShell shell = createTemplatedSliderApplication(
-        APPLICATION_NAME,
-        APP_TEMPLATE3,
-        APP_RESOURCE)
+    File launchReportFile = createAppReportFile();
 
+    SliderShell shell = createTemplatedSliderApplication(
+        APPLICATION_NAME, APP_TEMPLATE3, APP_RESOURCE,
+        [],
+        launchReportFile)
     logShell(shell)
 
-    ensureApplicationIsUp(APPLICATION_NAME)
+    def appId = ensureYarnApplicationIsUp(launchReportFile)
     expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 3)
     sleep(1000 * 20)
     def cd = execStatus(APPLICATION_NAME)
     assert cd.statistics[COMMAND_LOGGER]["containers.requested"] >= 3
-
-    assert isApplicationUp(APPLICATION_NAME), 'App is not running.'
-
+    assertInYarnState(appId, YarnApplicationState.RUNNING)
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e7df654f/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
index 3847e3f..d5be0f8 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
@@ -20,11 +20,13 @@ package org.apache.slider.funtest.lifecycle
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
 import org.apache.slider.funtest.framework.AgentCommandTestBase
 import org.apache.slider.funtest.framework.FuntestProperties
+import org.apache.slider.funtest.framework.SliderShell
 import org.junit.After
 import org.junit.Test
 
@@ -50,20 +52,24 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
     }
 
     cleanup(APPLICATION_NAME)
-    def shell = createTemplatedSliderApplication(APPLICATION_NAME,
-        APP_TEMPLATE2,
-        APP_RESOURCE)
 
+    File launchReportFile = createAppReportFile();
+    SliderShell shell = createTemplatedSliderApplication(
+        APPLICATION_NAME,
+        APP_TEMPLATE2,
+        APP_RESOURCE,
+        [],
+        launchReportFile)
     logShell(shell)
 
-    ensureApplicationIsUp(APPLICATION_NAME)
+    def appId = ensureYarnApplicationIsUp(launchReportFile)
 
     expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 2)
     sleep(1000 * 20)
     assert isApplicationUp(APPLICATION_NAME), 'App is not running.'
     def cd = expectContainersLive(APPLICATION_NAME, COMMAND_LOGGER, 1)
     assert cd.statistics[COMMAND_LOGGER]["containers.requested"] >= 2
-    assert isApplicationUp(APPLICATION_NAME), 'App is not running.'
+    assertInYarnState(appId, YarnApplicationState.RUNNING)
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e7df654f/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
index 4dcbed0..5087c06 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentLaunchFailureIT.groovy
@@ -20,6 +20,8 @@ package org.apache.slider.funtest.lifecycle
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
+import org.apache.slider.server.appmaster.SliderAppMaster
+
 import static org.apache.slider.api.InternalKeys.*
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.params.Arguments
@@ -65,33 +67,33 @@ public class AgentLaunchFailureIT extends AgentCommandTestBase
         APP_RESOURCE2,
         [
             ARG_OPTION, CHAOS_MONKEY_ENABLED, "true",
+            ARG_OPTION, CHAOS_MONKEY_INTERVAL_SECONDS, "60",
             ARG_OPTION, CHAOS_MONKEY_PROBABILITY_AM_LAUNCH_FAILURE, 
              Integer.toString(PROBABILITY_PERCENT_100),
         ],
         launchReportFile)
 
-    shell.dumpOutput();
     assert launchReportFile.exists()
     assert launchReportFile.size() > 0
     def launchReport = maybeLoadAppReport(launchReportFile)
     assert launchReport;
     assert launchReport.applicationId;
-    def report = maybeLookupFromLaunchReport(launchReportFile)
-    assert report;
-    ensureApplicationIsUp(CLUSTER)
-
-    //stop
-    freeze(0, CLUSTER,
-        [
-            ARG_FORCE,
-            ARG_WAIT, Integer.toString(FREEZE_WAIT_TIME),
-            ARG_MESSAGE, "final-shutdown"
-        ])
-
-    destroy(0, CLUSTER)
 
-    //cluster now missing
-    exists(EXIT_UNKNOWN_INSTANCE, CLUSTER)
+    // spin expecting failure
+    def appId = launchReport.applicationId
+    sleep(5000)
+    describe("Awaiting failure")
+    try {
+      ensureYarnApplicationIsUp(appId)
+      fail("application is up")
+    } catch (AssertionError e) {
+      if(!e.toString().contains(SliderAppMaster.E_TRIGGERED_LAUNCH_FAILURE)) {
+        throw e;
+      }
+    }
+    def sar = lookupApplication(appId)
+    log.info(sar.toString())
+    assert sar.diagnostics.contains(SliderAppMaster.E_TRIGGERED_LAUNCH_FAILURE)
 
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e7df654f/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
index 16e65fa..b5ee23d 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentRegistryIT.groovy
@@ -23,6 +23,7 @@ import groovy.util.logging.Slf4j
 import org.apache.hadoop.registry.client.binding.RegistryUtils
 import org.apache.hadoop.registry.client.types.Endpoint
 import org.apache.hadoop.registry.client.types.ServiceRecord
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.SliderKeys
 import org.apache.slider.common.params.Arguments
@@ -59,19 +60,17 @@ public class AgentRegistryIT extends AgentCommandTestBase
   @Test
   public void testAgentRegistry() throws Throwable {
     describe("Create a 0-role cluster and make registry queries against it")
-
-    // sanity check to verify the config is correct
-    assert clusterFS.uri.scheme != "file"
-
     def clusterpath = buildClusterPath(CLUSTER)
-    assert !clusterFS.exists(clusterpath)
+    File launchReportFile = createAppReportFile();
     SliderShell shell = createTemplatedSliderApplication(CLUSTER,
         APP_TEMPLATE,
-        APP_RESOURCE2)
+        APP_RESOURCE2,
+        [],
+        launchReportFile)
 
     logShell(shell)
 
-    ensureApplicationIsUp(CLUSTER)
+    def appId = ensureYarnApplicationIsUp(launchReportFile)
 
     //at this point the cluster should exist.
     assertPathExists(
@@ -124,15 +123,16 @@ public class AgentRegistryIT extends AgentCommandTestBase
     //stop
     freeze(0, CLUSTER,
         [
-            ARG_FORCE,
             ARG_WAIT, Integer.toString(FREEZE_WAIT_TIME),
             ARG_MESSAGE, "final-shutdown"
         ])
 
+    assertInYarnState(appId, YarnApplicationState.FINISHED)
     destroy(0, CLUSTER)
 
     //cluster now missing
     exists(EXIT_UNKNOWN_INSTANCE, CLUSTER)
 
+
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e7df654f/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentIT.groovy
index 488173b..5a5b964 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentIT.groovy
@@ -20,6 +20,7 @@ package org.apache.slider.funtest.lifecycle
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
@@ -53,13 +54,15 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
     assumeAgentTestsEnabled()
 
     cleanup(APPLICATION_NAME)
+    File launchReportFile = createAppReportFile();
     SliderShell shell = createTemplatedSliderApplication(APPLICATION_NAME,
         APP_TEMPLATE,
-        APP_RESOURCE)
-
+        APP_RESOURCE,
+        [],
+        launchReportFile)
     logShell(shell)
 
-    ensureApplicationIsUp(APPLICATION_NAME)
+    def appId = ensureYarnApplicationIsUp(launchReportFile)
 
     //flex
     slider(EXIT_SUCCESS,
@@ -107,8 +110,8 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
             APPLICATION_NAME,
             ARG_GETEXP,
             "container_log_dirs"])
-    if(!containsString(shell, "\"tag\" : \"COMMAND_LOGGER\"", 2)
-    || !containsString(shell, "\"level\" : \"component\"", 2)) {
+    if (!containsString(shell, "\"tag\" : \"COMMAND_LOGGER\"", 2)
+        || !containsString(shell, "\"level\" : \"component\"", 2)) {
       logShell(shell)
       assert fail("Should list 2 entries for log folders")
     }
@@ -159,6 +162,6 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
       assert fail("Should have exported cl-site")
     }
 
-    assert isApplicationUp(APPLICATION_NAME), 'App is not running.'
+    assertInYarnState(appId,  YarnApplicationState.RUNNING)
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e7df654f/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentQueueAndLabelsIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentQueueAndLabelsIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentQueueAndLabelsIT.groovy
index ec999f5..ee418dc 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentQueueAndLabelsIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AppsThroughAgentQueueAndLabelsIT.groovy
@@ -20,6 +20,7 @@ package org.apache.slider.funtest.lifecycle
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
@@ -80,15 +81,16 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
     assumeLabelsRedAndBlueAdded()
 
     cleanup(APPLICATION_NAME)
-    SliderShell shell = createTemplatedSliderApplication(APPLICATION_NAME,
+    File launchReportFile = createAppReportFile();
+    SliderShell shell = createTemplatedSliderApplication(
+        APPLICATION_NAME,
         APP_TEMPLATE,
         APP_RESOURCE4,
-        [ARG_QUEUE, TARGET_QUEUE]
-    )
-
+        [ARG_QUEUE, TARGET_QUEUE],
+        launchReportFile)
     logShell(shell)
 
-    ensureApplicationIsUp(APPLICATION_NAME)
+    def appId = ensureYarnApplicationIsUp(launchReportFile)
 
     expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 1 )
     expectContainersLive(APPLICATION_NAME, COMMAND_LOGGER, 1)
@@ -100,18 +102,18 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
             APPLICATION_NAME,
             ARG_COMPONENT,
             COMMAND_LOGGER,
-            "3"])
+            "3"
+        ])
 
-    // sleep till the new instance starts
-    ensureApplicationIsUp(APPLICATION_NAME)
+    // spin till the flexed instance starts
+    ensureYarnApplicationIsUp(appId)
     expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 3)
 
 
     sleep(1000 * 20)
     def cd = execStatus(APPLICATION_NAME)
     assert cd.statistics[COMMAND_LOGGER]["containers.requested"] >= 3
-
-    assert isApplicationUp(APPLICATION_NAME), 'App is not running.'
+    assertInYarnState(appId, YarnApplicationState.RUNNING)
   }
 
 


[09/13] git commit: SLIDER-570: AgentFailuresIT

Posted by st...@apache.org.
SLIDER-570: AgentFailuresIT


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/6e171c77
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/6e171c77
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/6e171c77

Branch: refs/heads/develop
Commit: 6e171c77f11bf196f56b05d34c6ffb4523476fbd
Parents: e7df654
Author: Steve Loughran <st...@apache.org>
Authored: Thu Oct 30 23:06:25 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Oct 31 11:07:50 2014 +0000

----------------------------------------------------------------------
 .../org/apache/slider/funtest/framework/CommandTestBase.groovy   | 4 ++--
 .../org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy   | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e171c77/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
index 0a7b295..fc74129 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
@@ -1051,14 +1051,14 @@ abstract class CommandTestBase extends SliderTestUtils {
     return requested
   }
 
-  boolean hasRequestedContainerCountReached(Map<String, String> args) {
+  Outcome hasRequestedContainerCountReached(Map<String, String> args) {
     String application = args['application']
     String role = args['role']
     int expectedCount = args['limit'].toInteger();
 
     int requestedCount = queryRequestedCount(application, role)
     log.debug("requested count = $requestedCount; expected=$expectedCount")
-    return requestedCount >= expectedCount
+    return Outcome.fromBool(requestedCount >= expectedCount)
   }
 
   void expectContainerRequestedCountReached(String application, String role, int limit) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6e171c77/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
index d5be0f8..0410881 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentFailuresIT.groovy
@@ -66,7 +66,7 @@ implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
 
     expectContainerRequestedCountReached(APPLICATION_NAME, COMMAND_LOGGER, 2)
     sleep(1000 * 20)
-    assert isApplicationUp(APPLICATION_NAME), 'App is not running.'
+    assertInYarnState(appId, YarnApplicationState.RUNNING)
     def cd = expectContainersLive(APPLICATION_NAME, COMMAND_LOGGER, 1)
     assert cd.statistics[COMMAND_LOGGER]["containers.requested"] >= 2
     assertInYarnState(appId, YarnApplicationState.RUNNING)