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/05/14 16:19:40 UTC

svn commit: r1594585 [1/2] - in /incubator/slider/trunk: slider-core/src/main/java/org/apache/slider/common/ slider-core/src/main/java/org/apache/slider/common/params/ slider-core/src/main/java/org/apache/slider/common/tools/ slider-core/src/main/java/...

Author: stevel
Date: Wed May 14 14:19:38 2014
New Revision: 1594585

URL: http://svn.apache.org/r1594585
Log:
SLIDER-33 Move masterless AM tests from HBase provider to slider-core

Added:
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionExists.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionExists.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionList.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionList.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionStatus.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionStatus.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionVersion.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionVersion.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeCommands.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeCommands.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeThawMasterlessAM.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeThawMasterlessAM.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeUnknownCluster.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeUnknownCluster.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestThawUnknownCluster.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestThawUnknownCluster.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestBadAMHeap.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestBadArguments.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestBadYarnQueue.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestBuildStandaloneAM.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/build/TestBuildClusterM1W5.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestDestroyMasterlessAM.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestDestroyMasterlessAM.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestKillMasterlessAM.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestKillMasterlessAM.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneBadClusterName.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestBadClusterName.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestHBaseMasterWithBadHeap.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestHBaseMaster.groovy
Removed:
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/build/TestBuildClusterM1W5.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeCommands.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeThawMasterlessAM.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeUnknownCluster.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestThawUnknownCluster.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestBadArguments.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestBadClusterName.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestCreateDuplicateLiveCluster.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestDestroyMasterlessAM.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestKillMasterlessAM.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestRecreateMasterlessAM.groovy
Modified:
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/params/ActionStatusArgs.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/build/InstanceBuilder.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/persist/ConfPersister.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/persist/InstancePaths.java
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/AgentMiniClusterTestBase.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/HBaseMiniClusterTestBase.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/archives/TestFreezeThawClusterFromArchive.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/archives/TestLiveClusterFromArchive.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/build/TestBuildThawClusterM1W1.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/failures/TestFailedRegionService.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/failures/TestFailureThreshold.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/failures/TestKilledHBaseMaster.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/flexing/TestClusterFlex1To1.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/flexing/TestClusterFlex1To2.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/flexing/TestClusterFlex2DownTo1.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/flexing/TestClusterFlex2To5.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/flexing/TestClusterFlexDownToZero.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/flexing/TestHMasterFlex1To2.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeReconfigureThawLiveRegionService.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeThawLiveRegionService.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/Test2Master2RS.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestHBaseMaster.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestHBaseMasterOnHDFS.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestLiveRegionServiceOnHDFS.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestTwoLiveClusters.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestSliderConfDirToMasterlessAM.groovy

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java?rev=1594585&r1=1594584&r2=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java Wed May 14 14:19:38 2014
@@ -137,6 +137,7 @@ public interface SliderKeys extends Slid
 
   String TMP_LOGDIR_PREFIX = "/tmp/slider-";
   String TMP_DIR_PREFIX = "tmp";
+  String AM_DIR_PREFIX = "appmaster";
   
   String SLIDER_JAR = "slider.jar";
   String JCOMMANDER_JAR = "jcommander.jar";

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/params/ActionStatusArgs.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/params/ActionStatusArgs.java?rev=1594585&r1=1594584&r2=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/params/ActionStatusArgs.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/params/ActionStatusArgs.java Wed May 14 14:19:38 2014
@@ -32,9 +32,13 @@ public class ActionStatusArgs extends Ab
 
   @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT},
              description = "Output file for the configuration data")
-  private String output;
+  public String output;
 
   public String getOutput() {
     return output;
   }
+
+  public void setOutput(String output) {
+    this.output = output;
+  }
 }

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java?rev=1594585&r1=1594584&r2=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java Wed May 14 14:19:38 2014
@@ -153,11 +153,11 @@ public class CoreFileSystem {
    */
   public void createClusterDirectories(InstancePaths instancePaths) throws
       IOException, SliderException {
-    Path clusterDirectory = instancePaths.instanceDir;
+    Path instanceDir = instancePaths.instanceDir;
 
-    verifyDirectoryNonexistent(clusterDirectory);
+    verifyDirectoryNonexistent(instanceDir);
     FsPermission clusterPerms = getInstanceDirectoryPermissions();
-    createWithPermissions(clusterDirectory, clusterPerms);
+    createWithPermissions(instanceDir, clusterPerms);
     createWithPermissions(instancePaths.snapshotConfPath, clusterPerms);
     createWithPermissions(instancePaths.generatedConfPath, clusterPerms);
     createWithPermissions(instancePaths.historyPath, clusterPerms);

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/build/InstanceBuilder.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/build/InstanceBuilder.java?rev=1594585&r1=1594584&r2=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/build/InstanceBuilder.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/build/InstanceBuilder.java Wed May 14 14:19:38 2014
@@ -230,9 +230,9 @@ public class InstanceBuilder {
    * @param appconfdir
    */
   public void persist(Path appconfdir) throws
-                                       IOException,
+      IOException,
       SliderException,
-                                       LockAcquireFailedException {
+      LockAcquireFailedException {
     coreFS.createClusterDirectories(instancePaths);
     ConfPersister persister =
       new ConfPersister(coreFS, getInstanceDir());

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/persist/ConfPersister.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/persist/ConfPersister.java?rev=1594585&r1=1594584&r2=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/persist/ConfPersister.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/persist/ConfPersister.java Wed May 14 14:19:38 2014
@@ -232,7 +232,7 @@ public class ConfPersister {
 
 
   private void maybeExecLockHeldAction(LockHeldAction action) throws
-                                                              IOException,
+      IOException,
       SliderException {
     if (action != null) {
       action.execute();
@@ -247,9 +247,9 @@ public class ConfPersister {
    * @throws LockAcquireFailedException the lock could not be acquired
    */
   public void save(AggregateConf conf, LockHeldAction action) throws
-                                        IOException,
+      IOException,
       SliderException,
-                                        LockAcquireFailedException {
+      LockAcquireFailedException {
     acquireWritelock();
     try {
       saveConf(conf);
@@ -268,10 +268,10 @@ public class ConfPersister {
    * @throws LockAcquireFailedException the lock could not be acquired
    */
   public void load(AggregateConf conf) throws
-                                       FileNotFoundException,
-                                        IOException,
+      FileNotFoundException,
+      IOException,
       SliderException,
-                                        LockAcquireFailedException {
+      LockAcquireFailedException {
     boolean owner = acquireReadLock();
     try {
       loadConf(conf);

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/persist/InstancePaths.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/persist/InstancePaths.java?rev=1594585&r1=1594584&r2=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/persist/InstancePaths.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/persist/InstancePaths.java Wed May 14 14:19:38 2014
@@ -44,7 +44,7 @@ public class InstancePaths {
     historyPath = new Path(instanceDir, SliderKeys.HISTORY_DIR_NAME);
     dataPath = new Path(instanceDir, SliderKeys.DATA_DIR_NAME);
     tmpPath = new Path(instanceDir, SliderKeys.TMP_DIR_PREFIX);
-    tmpPathAM = new Path(tmpPath, "appmaster");
+    tmpPathAM = new Path(tmpPath, SliderKeys.AM_DIR_PREFIX);
   }
 
   @Override

Modified: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/AgentMiniClusterTestBase.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/AgentMiniClusterTestBase.groovy?rev=1594585&r1=1594584&r2=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/AgentMiniClusterTestBase.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/AgentMiniClusterTestBase.groovy Wed May 14 14:19:38 2014
@@ -28,6 +28,8 @@ import org.apache.slider.providers.agent
 import org.apache.slider.test.YarnZKMiniClusterTestBase
 import org.junit.BeforeClass
 
+import javax.swing.ListModel
+
 /**
  * test base for agent clusters
  */
@@ -35,10 +37,10 @@ import org.junit.BeforeClass
 @Slf4j
 public abstract class AgentMiniClusterTestBase
     extends YarnZKMiniClusterTestBase {
-  private static File agentConf
-  private static File agentDef
-  private static File imagePath
-
+  protected static File agentConf
+  protected static File agentDef
+  protected static File imagePath
+  protected static Map<String, String> agentDefOptions 
 
   @BeforeClass
   public static void createSubConfFiles() {
@@ -50,7 +52,12 @@ public abstract class AgentMiniClusterTe
     agentDef.createNewFile()
     File slider_dir = new File(new File(".").absoluteFile, "src/test/python");
     imagePath = new File(slider_dir, "appdef_1.zip")
+    agentDefOptions = [
+        (AgentKeys.APP_DEF)   : imagePath.toURI().toString(),
+        (AgentKeys.AGENT_CONF): agentConf.toURI().toString()
+    ]
   }
+
   @Override
   public String getTestConfigurationPath() {
     return "src/main/resources/" + AgentKeys.CONF_RESOURCE;
@@ -104,17 +111,37 @@ public abstract class AgentMiniClusterTe
       int size,
       boolean deleteExistingData,
       boolean blockUntilRunning) {
+    List<String> args = [];
+    return createMasterlessAMWithArgs(
+        clustername,
+        args,
+        deleteExistingData,
+        blockUntilRunning)
+  }
+
+/**
+ * Create an AM without a master
+ * @param clustername AM name
+ * @param extraArgs extra arguments
+ * @param size # of nodes
+ * @param deleteExistingData should any existing cluster data be deleted
+ * @param blockUntilRunning block until the AM is running
+ * @return launcher which will have executed the command.
+ */
+  public ServiceLauncher<SliderClient> createMasterlessAMWithArgs(
+      String clustername,
+      List<String> extraArgs,
+      boolean deleteExistingData,
+      boolean blockUntilRunning) {
+    if (hdfsCluster) {
+      fail("Agent tests do not (currently) work with mini HDFS cluster")
+    }
     return createCluster(clustername,
         [:],
-        [
-
-        ],
+        extraArgs,
         deleteExistingData,
         blockUntilRunning,
-        [
-            (AgentKeys.APP_DEF): imagePath.toURI().toString(),
-            (AgentKeys.AGENT_CONF): agentConf.toURI().toString()
-        ])
+        agentDefOptions)
   }
 
-}
+}
\ No newline at end of file

Copied: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionExists.groovy (from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionExists.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionExists.groovy?p2=incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionExists.groovy&p1=incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionExists.groovy&r1=1594525&r2=1594585&rev=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionExists.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionExists.groovy Wed May 14 14:19:38 2014
@@ -16,34 +16,36 @@
  *  limitations under the License.
  */
 
-package org.apache.slider.providers.hbase.actions
+package org.apache.slider.agent.actions
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.slider.core.main.LauncherExitCodes
-import org.apache.slider.core.exceptions.UnknownApplicationInstanceException
-import org.apache.slider.common.params.Arguments
-import org.apache.slider.common.params.SliderActions
-import org.apache.slider.client.SliderClient
-import org.apache.slider.providers.hbase.minicluster.HBaseMiniClusterTestBase
 import org.apache.hadoop.yarn.api.records.ApplicationReport
 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.Arguments
+import org.apache.slider.common.params.SliderActions
+import org.apache.slider.core.exceptions.UnknownApplicationInstanceException
+import org.apache.slider.core.main.LauncherExitCodes
 import org.apache.slider.core.main.ServiceLauncher
+import org.apache.slider.test.SliderTestUtils
+import org.junit.Assert
 import org.junit.Before
 import org.junit.Test
 
 /**
- * Test of RM creation. This is so the later test's prereq's can be met
+ * existence tests
  */
 @CompileStatic
 @Slf4j
 
-class TestActionExists extends HBaseMiniClusterTestBase {
+class TestActionExists extends AgentMiniClusterTestBase {
 
   @Before
   public void setup() {
     super.setup()
-    createMiniCluster("TestActionExists", getConfiguration(), 1, false)
+    createMiniCluster("TestActionExists", configuration, 1, false)
   }
   
   @Test
@@ -60,7 +62,7 @@ class TestActionExists extends HBaseMini
           Arguments.ARG_MANAGER, RMAddr
           ],
       )
-      fail("expected an exception, got a status code "+ launcher.serviceExitCode)
+      Assert.fail("expected an exception, got a status code "+ launcher.serviceExitCode)
     } catch (UnknownApplicationInstanceException e) {
       
     }
@@ -86,7 +88,7 @@ class TestActionExists extends HBaseMini
           Arguments.ARG_MANAGER, RMAddr
           ],
       )
-    assertSucceeded(launcher)
+    SliderTestUtils.assertSucceeded(launcher)
 
     //and when cluster is running
     launcher = launchClientAgainstMiniMR(
@@ -101,7 +103,7 @@ class TestActionExists extends HBaseMini
           ],
       )
 
-    assertSucceeded(launcher)
+    SliderTestUtils.assertSucceeded(launcher)
     
     // assert that the cluster exists
 

Copied: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionList.groovy (from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionList.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionList.groovy?p2=incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionList.groovy&p1=incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionList.groovy&r1=1594525&r2=1594585&rev=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionList.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionList.groovy Wed May 14 14:19:38 2014
@@ -16,37 +16,50 @@
  *  limitations under the License.
  */
 
-package org.apache.slider.providers.hbase.actions
+package org.apache.slider.agent.actions
 
 import groovy.util.logging.Slf4j
-import org.apache.slider.core.exceptions.UnknownApplicationInstanceException
-import org.apache.slider.common.params.Arguments
-import org.apache.slider.common.params.SliderActions
-import org.apache.slider.client.SliderClient
-import org.apache.slider.providers.hbase.minicluster.HBaseMiniClusterTestBase
 import org.apache.hadoop.yarn.api.records.ApplicationReport
 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.Arguments
+import org.apache.slider.common.params.SliderActions
+import org.apache.slider.core.exceptions.UnknownApplicationInstanceException
 import org.apache.slider.core.main.ServiceLauncher
 import org.junit.Before
 import org.junit.Test
 
 /**
- * Test of RM creation. This is so the later test's prereq's can be met
+ * Test List operations
  */
 @Slf4j
 
-class TestActionList extends HBaseMiniClusterTestBase {
+class TestActionList extends AgentMiniClusterTestBase {
 
   @Before
   public void setup() {
     super.setup()
-    createMiniCluster("testActionList", getConfiguration(), 1, false)
+    createMiniCluster("testActionList", configuration, 1, false)
   }
-  
+
+  /**
+   * This is a test suite to run the tests against a single cluster instance
+   * for faster test runs
+   * @throws Throwable
+   */
+
   @Test
+  public void testSuite() throws Throwable {
+    testListThisUserNoClusters()
+    testListAllUsersNoClusters()
+    testListLiveCluster()
+    testListMissingCluster()
+  }
+  
   public void testListThisUserNoClusters() throws Throwable {
     log.info("RM address = ${RMAddr}")
-    ServiceLauncher launcher = launchClientAgainstMiniMR(
+    ServiceLauncher<SliderClient> launcher = launchClientAgainstMiniMR(
         //config includes RM binding info
         new YarnConfiguration(miniCluster.config),
         //varargs list of command line params
@@ -56,13 +69,11 @@ class TestActionList extends HBaseMiniCl
         ]
     )
     assert launcher.serviceExitCode == 0
-    SliderClient sliderClient = (SliderClient) launcher.service
   }
   
-  @Test
   public void testListAllUsersNoClusters() throws Throwable {
     log.info("RM address = ${RMAddr}")
-    ServiceLauncher launcher = launchClientAgainstMiniMR(
+    ServiceLauncher<SliderClient> launcher = launchClientAgainstMiniMR(
         //config includes RM binding info
         new YarnConfiguration(miniCluster.config),
         //varargs list of command line params
@@ -74,12 +85,14 @@ class TestActionList extends HBaseMiniCl
     assert launcher.serviceExitCode == 0
   }
 
-  @Test
   public void testListLiveCluster() throws Throwable {
     //launch the cluster
     String clustername = "test_list_live_cluster"
-    ServiceLauncher launcher = createMasterlessAM(clustername, 0, true, false)
-    ApplicationReport report = waitForClusterLive((SliderClient) launcher.service)
+    ServiceLauncher<SliderClient> launcher = createMasterlessAM(clustername, 0, true, false)
+    addToTeardown(launcher)
+    //do the low level operations to get a better view of what is going on 
+    SliderClient sliderClient = launcher.service
+    waitForClusterLive(sliderClient)
 
     //now list
     launcher = launchClientAgainstMiniMR(
@@ -93,8 +106,7 @@ class TestActionList extends HBaseMiniCl
     assert launcher.serviceExitCode == 0
     //now look for the explicit sevice
     
-    //do the low level operations to get a better view of what is going on 
-    SliderClient sliderClient = launcher.service
+
     def serviceRegistryClient = sliderClient.YARNRegistryClient
     ApplicationReport instance = serviceRegistryClient.findInstance(clustername)
     assert instance != null
@@ -112,15 +124,10 @@ class TestActionList extends HBaseMiniCl
 
   }
 
-
-
-  @Test
   public void testListMissingCluster() throws Throwable {
-    describe("create exec the status command against an unknown cluster")
-    //launch fake master
-    //launch the cluster
-    //exec the status command
-    ServiceLauncher launcher
+    describe("exec the status command against an unknown cluster")
+
+    ServiceLauncher<SliderClient> launcher
     try {
       launcher = launchClientAgainstMiniMR(
           //config includes RM binding info

Copied: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionStatus.groovy (from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionStatus.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionStatus.groovy?p2=incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionStatus.groovy&p1=incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionStatus.groovy&r1=1594525&r2=1594585&rev=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionStatus.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionStatus.groovy Wed May 14 14:19:38 2014
@@ -16,11 +16,14 @@
  *  limitations under the License.
  */
 
-package org.apache.slider.providers.hbase.actions
+package org.apache.slider.agent.actions
 
+import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
+import org.apache.slider.agent.AgentMiniClusterTestBase
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.api.ClusterDescription
+import org.apache.slider.common.params.SliderActions
 import org.apache.slider.core.exceptions.BadClusterStateException
 import org.apache.slider.core.exceptions.ErrorStrings
 import org.apache.slider.core.exceptions.UnknownApplicationInstanceException
@@ -28,19 +31,19 @@ import org.apache.slider.common.params.A
 import org.apache.slider.client.SliderClient
 import org.apache.slider.common.params.ActionStatusArgs
 import org.apache.slider.common.params.ClientArgs
-import org.apache.slider.providers.hbase.minicluster.HBaseMiniClusterTestBase
 import org.apache.hadoop.yarn.api.records.ApplicationReport
 import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.slider.core.main.LauncherExitCodes
 import org.apache.slider.core.main.ServiceLauncher
 import org.junit.Before
 import org.junit.Test
 
 /**
- * Test of RM creation. This is so the later test's prereq's can be met
+ * status operations
  */
-//@CompileStatic
+@CompileStatic
 @Slf4j
-class TestActionStatus extends HBaseMiniClusterTestBase {
+class TestActionStatus extends AgentMiniClusterTestBase {
 
   @Before
   public void setup() {
@@ -48,19 +51,26 @@ class TestActionStatus extends HBaseMini
     createMiniCluster("test_action_status", configuration, 1, false)
   }
 
+  /**
+   * This is a test suite to run the tests against a single cluster instance
+   * for faster test runs
+   * @throws Throwable
+   */
+
   @Test
+  public void testSuite() throws Throwable {
+    testStatusLiveCluster()
+    testStatusMissingCluster()
+  }
+
   public void testStatusMissingCluster() throws Throwable {
     describe("create exec the status command against an unknown cluster")
-    //launch fake master
-    //launch the cluster
-    //exec the status command
+
     try {
-      ServiceLauncher launcher = launchClientAgainstMiniMR(
-          //config includes RM binding info
+      ServiceLauncher<SliderClient> launcher = launchClientAgainstMiniMR(
           new YarnConfiguration(miniCluster.config),
-          //varargs list of command line params
           [
-              ClientArgs.ACTION_STATUS,
+              SliderActions.ACTION_STATUS,
               "test_status_missing_cluster",
               Arguments.ARG_MANAGER, RMAddr
           ]
@@ -72,25 +82,22 @@ class TestActionStatus extends HBaseMini
 
   }
   
-  @Test
   public void testStatusLiveCluster() throws Throwable {
     describe("create a live cluster then exec the status command")
-    //launch fake master
     String clustername = "test_status_live_cluster"
     
     //launch the cluster
-    ServiceLauncher launcher = createMasterlessAM(clustername, 0, true, false)
+    ServiceLauncher<SliderClient> launcher = createMasterlessAM(clustername, 0, true, false)
 
-    ApplicationReport report = waitForClusterLive(launcher.service)
+    SliderClient sliderClient = launcher.service
+    ApplicationReport report = waitForClusterLive(sliderClient)
 
-    //do the low level operations to get a better view of what is going on 
-    SliderClient sliderClient = (SliderClient) launcher.service
 
     //now look for the explicit sevice
 
     ActionStatusArgs statusArgs = new ActionStatusArgs()
     int status = sliderClient.actionStatus(clustername, statusArgs)
-    assert status == SliderExitCodes.EXIT_SUCCESS
+    assert 0 == status
 
     //now exec the status command
     ServiceLauncher statusLauncher = launchClientAgainstMiniMR(
@@ -98,7 +105,7 @@ class TestActionStatus extends HBaseMini
         new YarnConfiguration(miniCluster.config),
         //varargs list of command line params
         [
-            ClientArgs.ACTION_STATUS,
+            SliderActions.ACTION_STATUS,
             clustername,
             Arguments.ARG_MANAGER, RMAddr,
         ]
@@ -121,7 +128,7 @@ class TestActionStatus extends HBaseMini
         new YarnConfiguration(miniCluster.config),
         //varargs list of command line params
         [
-            ClientArgs.ACTION_STATUS,
+            SliderActions.ACTION_STATUS,
             clustername,
             Arguments.ARG_MANAGER, RMAddr,
             Arguments.ARG_OUTPUT, path
@@ -140,7 +147,8 @@ class TestActionStatus extends HBaseMini
       status = sliderClient.actionStatus(clustername, new ActionStatusArgs())
       fail("expected an exception, but got the status $status")
     } catch (BadClusterStateException e) {
-      assert e.toString().contains(ErrorStrings.E_APPLICATION_NOT_RUNNING)
+      assertExceptionDetails(e, SliderExitCodes.EXIT_BAD_STATE,
+          ErrorStrings.E_APPLICATION_NOT_RUNNING)
     }
   }
 

Copied: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionVersion.groovy (from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionVersion.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionVersion.groovy?p2=incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionVersion.groovy&p1=incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionVersion.groovy&r1=1594525&r2=1594585&rev=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/actions/TestActionVersion.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/actions/TestActionVersion.groovy Wed May 14 14:19:38 2014
@@ -16,7 +16,7 @@
  *  limitations under the License.
  */
 
-package org.apache.slider.providers.hbase.actions
+package org.apache.slider.agent.actions
 
 import groovy.util.logging.Slf4j
 import org.apache.slider.common.params.SliderActions
@@ -33,11 +33,6 @@ import org.junit.Test
 
 class TestActionVersion extends YarnMiniClusterTestBase {
 
-  @Before
-  public void setup() {
-    super.setup()
-  }
-  
   @Test
   public void testVersion() throws Throwable {
     

Copied: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeCommands.groovy (from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeCommands.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeCommands.groovy?p2=incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeCommands.groovy&p1=incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeCommands.groovy&r1=1594525&r2=1594585&rev=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeCommands.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeCommands.groovy Wed May 14 14:19:38 2014
@@ -16,18 +16,19 @@
  *  limitations under the License.
  */
 
-package org.apache.slider.providers.hbase.minicluster.freezethaw
+package org.apache.slider.agent.freezethaw
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.slider.core.main.LauncherExitCodes
-import org.apache.slider.core.exceptions.SliderException
+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.Arguments
 import org.apache.slider.common.params.SliderActions
-import org.apache.slider.client.SliderClient
-import org.apache.slider.providers.hbase.minicluster.HBaseMiniClusterTestBase
-import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.slider.core.exceptions.SliderException
+import org.apache.slider.core.main.LauncherExitCodes
 import org.apache.slider.core.main.ServiceLauncher
+import org.apache.slider.test.YarnMiniClusterTestBase
 import org.junit.Test
 
 /**
@@ -37,44 +38,50 @@ import org.junit.Test
 @CompileStatic
 @Slf4j
 
-class TestFreezeCommands extends HBaseMiniClusterTestBase {
+class TestFreezeCommands extends AgentMiniClusterTestBase {
 
   @Test
   public void testFreezeCommands() throws Throwable {
     String clustername = "test_freeze_commands"
-    YarnConfiguration conf = getConfiguration()
-    createMiniCluster(clustername, conf, 1, 1, 1, true, true)
+    YarnConfiguration conf = configuration
+    createMiniCluster(clustername, conf, 1, 1, 1, true, false)
 
     describe "create a masterless AM, freeze it, try to freeze again"
 
-    ServiceLauncher launcher = createMasterlessAM(clustername, 0, true, true);
-    addToTeardown(launcher.service as SliderClient);
+    ServiceLauncher<SliderClient> launcher = createMasterlessAM(
+        clustername,
+        0,
+        true,
+        true);
+    addToTeardown(launcher.service);
+
 
-    
     log.info("ListOp")
     assertSucceeded(execSliderCommand(conf,
-              [SliderActions.ACTION_LIST,clustername]))
-    
+        [SliderActions.ACTION_LIST, clustername]))
+
     log.info("First Freeze command");
     ServiceLauncher freezeCommand = execSliderCommand(conf,
-                          [SliderActions.ACTION_FREEZE, clustername,
-                            Arguments.ARG_WAIT, waitTimeArg]);
+        [
+            SliderActions.ACTION_FREEZE, clustername,
+            Arguments.ARG_WAIT, waitTimeArg
+        ]);
     assertSucceeded(freezeCommand)
 
     log.info("Second Freeze command");
 
-    ServiceLauncher freeze2 = execSliderCommand(conf,
-                                [
-                                    SliderActions.ACTION_FREEZE, clustername,
-                                    Arguments.ARG_WAIT, waitTimeArg
-                                ]);
+    ServiceLauncher<SliderClient> freeze2 = execSliderCommand(conf,
+        [
+            SliderActions.ACTION_FREEZE, clustername,
+            Arguments.ARG_WAIT, waitTimeArg
+        ]);
     assertSucceeded(freeze2)
 
     log.info("First Exists");
 
     //assert there is no running cluster
     try {
-      ServiceLauncher exists1 = launchClientAgainstMiniMR(
+      ServiceLauncher<SliderClient> exists1 = launchClientAgainstMiniMR(
           //config includes RM binding info
           new YarnConfiguration(miniCluster.config),
           [
@@ -82,7 +89,7 @@ class TestFreezeCommands extends HBaseMi
               Arguments.ARG_FILESYSTEM, fsDefaultName,
               Arguments.ARG_LIVE
           ],
-          )
+      )
       assert 0 != exists1.serviceExitCode;
     } catch (SliderException e) {
       assert e.exitCode == LauncherExitCodes.EXIT_FALSE;
@@ -97,36 +104,36 @@ class TestFreezeCommands extends HBaseMi
         Arguments.ARG_FILESYSTEM, fsDefaultName
     ]
     commands.addAll(extraCLIArgs)
-    
+
     ServiceLauncher thawCommand = execSliderCommand(conf, commands);
     assertSucceeded(thawCommand)
     assertSucceeded(execSliderCommand(conf,
-                  [SliderActions.ACTION_LIST, clustername]))
+        [SliderActions.ACTION_LIST, clustername]))
     assertSucceeded(execSliderCommand(conf,
-                  [SliderActions.ACTION_EXISTS, clustername]))
+        [SliderActions.ACTION_EXISTS, clustername]))
 
     log.info("Freeze 3");
 
-    ServiceLauncher freeze3 = execSliderCommand(conf,
-                [
-                    SliderActions.ACTION_FREEZE, clustername,
-                    Arguments.ARG_WAIT, waitTimeArg
-                ]);
+    ServiceLauncher<SliderClient> freeze3 = execSliderCommand(conf,
+        [
+            SliderActions.ACTION_FREEZE, clustername,
+            Arguments.ARG_WAIT, waitTimeArg
+        ]);
     assertSucceeded(freeze3)
 
     log.info("thaw2");
-    ServiceLauncher thaw2 = execSliderCommand(conf,
+    ServiceLauncher<SliderClient> thaw2 = execSliderCommand(conf,
         commands);
     assert 0 == thaw2.serviceExitCode;
     assertSucceeded(thaw2)
 
     try {
       log.info("thaw3 - should fail");
-      ServiceLauncher thaw3 = execSliderCommand(conf,
+      ServiceLauncher<SliderClient> thaw3 = execSliderCommand(conf,
           commands);
       assert 0 != thaw3.serviceExitCode;
     } catch (SliderException e) {
-      assertFailureClusterInUse(e);
+      YarnMiniClusterTestBase.assertFailureClusterInUse(e);
     }
 
     //destroy should fail
@@ -134,7 +141,7 @@ class TestFreezeCommands extends HBaseMi
     log.info("destroy1");
 
     try {
-      ServiceLauncher destroy1 = execSliderCommand(conf,
+      ServiceLauncher<SliderClient> destroy1 = execSliderCommand(conf,
           [
               SliderActions.ACTION_DESTROY, clustername,
               Arguments.ARG_FILESYSTEM, fsDefaultName
@@ -145,24 +152,24 @@ class TestFreezeCommands extends HBaseMi
       assertFailureClusterInUse(e);
     }
     log.info("freeze4");
-    
+
     //kill -19 the process to hang it, then force kill
-    killAM(SIGSTOP)
+    killAM(YarnMiniClusterTestBase.SIGSTOP)
 
-    ServiceLauncher freeze4 = execSliderCommand(conf,
-                                              [
-                                                  SliderActions.ACTION_FREEZE, clustername,
-                                                  Arguments.ARG_FORCE,
-                                                  Arguments.ARG_WAIT, waitTimeArg,
-                                              ]);
+    ServiceLauncher<SliderClient> freeze4 = execSliderCommand(conf,
+        [
+            SliderActions.ACTION_FREEZE, clustername,
+            Arguments.ARG_FORCE,
+            Arguments.ARG_WAIT, waitTimeArg,
+        ]);
     assertSucceeded(freeze4)
 
     log.info("destroy2");
-    ServiceLauncher destroy2 = execSliderCommand(conf,
-                                               [
-                                                   SliderActions.ACTION_DESTROY, clustername,
-                                                   Arguments.ARG_FILESYSTEM, fsDefaultName,
-                                               ]);
+    ServiceLauncher<SliderClient> destroy2 = execSliderCommand(conf,
+        [
+            SliderActions.ACTION_DESTROY, clustername,
+            Arguments.ARG_FILESYSTEM, fsDefaultName,
+        ]);
     assertSucceeded(destroy2)
 
   }

Copied: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeThawMasterlessAM.groovy (from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeThawMasterlessAM.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeThawMasterlessAM.groovy?p2=incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeThawMasterlessAM.groovy&p1=incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeThawMasterlessAM.groovy&r1=1594525&r2=1594585&rev=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeThawMasterlessAM.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeThawMasterlessAM.groovy Wed May 14 14:19:38 2014
@@ -16,27 +16,26 @@
  *  limitations under the License.
  */
 
-package org.apache.slider.providers.hbase.minicluster.freezethaw
+package org.apache.slider.agent.freezethaw
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.fs.FileSystem as HadoopFS
 import org.apache.hadoop.fs.Path
-import org.apache.slider.common.tools.SliderUtils
-import org.apache.slider.client.SliderClient
-import org.apache.slider.providers.hbase.minicluster.HBaseMiniClusterTestBase
 import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.slider.agent.AgentMiniClusterTestBase
+import org.apache.slider.client.SliderClient
+import org.apache.slider.common.tools.SliderUtils
 import org.apache.slider.core.main.ServiceLauncher
 import org.junit.Test
 
 /**
- * create masterless AMs and work with them. This is faster than
- * bringing up full clusters
+ * freeze and thaw an AM
  */
 @CompileStatic
 @Slf4j
 
-class TestFreezeThawMasterlessAM extends HBaseMiniClusterTestBase {
+class TestFreezeThawMasterlessAM extends AgentMiniClusterTestBase {
 
   File getConfDirFile() {
     return new File("target/TestFreezeThawMasterlessAM/conf")
@@ -50,18 +49,18 @@ class TestFreezeThawMasterlessAM extends
   @Test
   public void testFreezeThawMasterlessAM() throws Throwable {
     String clustername = "test_freeze_thaw_masterless_am"
-    YarnConfiguration conf = getConfiguration()
-    createMiniCluster(clustername, conf, 1, 1, 1, true, true)
+    YarnConfiguration conf = configuration
+    createMiniCluster(clustername, conf, 1, 1, 1, true, false)
     
     describe "create a masterless AM, freeze it, thaw it"
     //copy the confdir somewhere
-    Path resConfPath = new Path(getResourceConfDirURI())
+    Path resConfPath = new Path(resourceConfDirURI)
     Path tempConfPath = new Path(confDir)
     SliderUtils.copyDirectory(conf, resConfPath, tempConfPath, null)
 
 
-    ServiceLauncher launcher = createMasterlessAM(clustername, 0, true, true)
-    SliderClient sliderClient = (SliderClient) launcher.service
+    ServiceLauncher<SliderClient> launcher = createMasterlessAM(clustername, 0, true, true)
+    SliderClient sliderClient = launcher.service
     addToTeardown(sliderClient);
 
     assert 0 == clusterActionFreeze(sliderClient, clustername)
@@ -74,7 +73,7 @@ class TestFreezeThawMasterlessAM extends
     
     //now start the cluster
     ServiceLauncher launcher2 = thawCluster(clustername, [], true);
-    SliderClient newCluster = launcher.getService() as SliderClient
+    SliderClient newCluster = launcher2.service
     newCluster.getClusterDescription(clustername);
     
     //freeze

Copied: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeUnknownCluster.groovy (from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeUnknownCluster.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeUnknownCluster.groovy?p2=incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeUnknownCluster.groovy&p1=incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeUnknownCluster.groovy&r1=1594525&r2=1594585&rev=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestFreezeUnknownCluster.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestFreezeUnknownCluster.groovy Wed May 14 14:19:38 2014
@@ -16,14 +16,15 @@
  *  limitations under the License.
  */
 
-package org.apache.slider.providers.hbase.minicluster.freezethaw
+package org.apache.slider.agent.freezethaw
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.slider.core.exceptions.UnknownApplicationInstanceException
-import org.apache.slider.common.params.SliderActions
-import org.apache.slider.providers.hbase.minicluster.HBaseMiniClusterTestBase
 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.SliderActions
+import org.apache.slider.core.exceptions.UnknownApplicationInstanceException
 import org.apache.slider.core.main.ServiceLauncher
 import org.junit.Test
 
@@ -34,19 +35,18 @@ import org.junit.Test
 @CompileStatic
 @Slf4j
 
-class TestFreezeUnknownCluster extends HBaseMiniClusterTestBase {
+class TestFreezeUnknownCluster extends AgentMiniClusterTestBase {
 
   @Test
   public void testFreezeUnknownCluster() throws Throwable {
     String clustername = "test_start_unknown_cluster"
-    YarnConfiguration conf = getConfiguration()
+    YarnConfiguration conf = configuration
     createMiniCluster(clustername, conf, 1, true)
 
     describe "try to freeze a cluster that isn't defined"
 
-    //we are secretly picking up the RM details from the configuration file
     try {
-      ServiceLauncher command = execSliderCommand(conf,
+      ServiceLauncher<SliderClient>  command = execSliderCommand(conf,
                                                 [
                                                     SliderActions.ACTION_FREEZE,
                                                     "no-such-cluster"

Copied: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestThawUnknownCluster.groovy (from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestThawUnknownCluster.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestThawUnknownCluster.groovy?p2=incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestThawUnknownCluster.groovy&p1=incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestThawUnknownCluster.groovy&r1=1594525&r2=1594585&rev=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/freezethaw/TestThawUnknownCluster.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/freezethaw/TestThawUnknownCluster.groovy Wed May 14 14:19:38 2014
@@ -16,13 +16,14 @@
  *  limitations under the License.
  */
 
-package org.apache.slider.providers.hbase.minicluster.freezethaw
+package org.apache.slider.agent.freezethaw
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
+import org.apache.slider.client.SliderClient
 import org.apache.slider.core.exceptions.UnknownApplicationInstanceException
-import org.apache.slider.test.YarnZKMiniClusterTestBase
 import org.apache.slider.core.main.ServiceLauncher
+import org.apache.slider.test.YarnZKMiniClusterTestBase
 import org.junit.Test
 
 /**
@@ -37,12 +38,12 @@ class TestThawUnknownCluster extends Yar
   @Test
   public void testThawUnknownCluster() throws Throwable {
     String clustername = "test_thaw_unknown_cluster"
-    createMiniCluster(clustername, getConfiguration(), 1, true)
+    createMiniCluster(clustername, configuration, 1, true)
 
     describe "try to start a cluster that isn't defined"
 
     try {
-      ServiceLauncher launcher = thawCluster(clustername, [], true);
+      ServiceLauncher<SliderClient> launcher = thawCluster(clustername, [], true);
       fail("expected a failure, got ${launcher.serviceExitCode}")
     } catch (UnknownApplicationInstanceException e) {
       assert e.toString().contains(clustername)

Copied: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestBadAMHeap.groovy (from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestBadArguments.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestBadAMHeap.groovy?p2=incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestBadAMHeap.groovy&p1=incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestBadArguments.groovy&r1=1594525&r2=1594585&rev=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestBadArguments.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestBadAMHeap.groovy Wed May 14 14:19:38 2014
@@ -16,89 +16,45 @@
  *  limitations under the License.
  */
 
-package org.apache.slider.providers.hbase.minicluster.masterless
+package org.apache.slider.agent.standalone
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
+import org.apache.hadoop.yarn.api.records.ApplicationReport
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
+import org.apache.slider.agent.AgentMiniClusterTestBase
+import org.apache.slider.api.RoleKeys
+import org.apache.slider.client.SliderClient
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.SliderKeys
-import org.apache.slider.common.SliderXmlConfKeys
-import org.apache.slider.api.RoleKeys
-import org.apache.slider.providers.hbase.HBaseKeys
 import org.apache.slider.common.params.Arguments
-import org.apache.slider.client.SliderClient
-import org.apache.slider.providers.hbase.minicluster.HBaseMiniClusterTestBase
-import org.apache.hadoop.yarn.api.records.ApplicationReport
-import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.slider.core.main.ServiceLaunchException
 import org.apache.slider.core.main.ServiceLauncher
 import org.junit.Test
 
-import static HBaseKeys.PROVIDER_HBASE
-import static Arguments.ARG_PROVIDER
-
 @CompileStatic
 @Slf4j
 
-class TestBadArguments extends HBaseMiniClusterTestBase {
+class TestBadAMHeap extends AgentMiniClusterTestBase {
 
   @Test
   public void testBadAMHeap() throws Throwable {
     String clustername = "test_bad_am_heap"
-    createMiniCluster(clustername, getConfiguration(), 1, true)
-
-    describe "verify that bad Java heap options are picked up"
-
-    try {
-      ServiceLauncher launcher = createCluster(clustername,
-           [
-               (HBaseKeys.ROLE_MASTER): 0,
-               (HBaseKeys.ROLE_WORKER): 0,
-           ],
-           [
-               Arguments.ARG_COMP_OPT, SliderKeys.COMPONENT_AM, RoleKeys.JVM_HEAP, "invalid",
-               ARG_PROVIDER, PROVIDER_HBASE
-           ],
-           true,
-           false,
-           [:])
-      SliderClient sliderClient = (SliderClient) launcher.service
-      addToTeardown(sliderClient);
-
-      ApplicationReport report = waitForClusterLive(sliderClient)
-      assert report.yarnApplicationState == YarnApplicationState.FAILED
-      
-    } catch (ServiceLaunchException e) {
-      assertExceptionDetails(e, SliderExitCodes.EXIT_YARN_SERVICE_FAILED)
-    }
-    
-  }
-
-  /**
-   * Test disabled because YARN queues don't get validated in the mini cluster
-   * @throws Throwable
-   */
-  public void DisabledtestBadYarnQueue() throws Throwable {
-    String clustername = "test_bad_yarn_queue"
-    createMiniCluster(clustername, getConfiguration(), 1, true)
+    createMiniCluster(clustername, configuration, 1, true)
 
     describe "verify that bad Java heap options are picked up"
 
     try {
-      ServiceLauncher launcher = createCluster(clustername,
-           [
-               (HBaseKeys.ROLE_MASTER): 0,
-               (HBaseKeys.ROLE_WORKER): 0,
-           ],
-           [
-               Arguments.ARG_DEFINE,
-               SliderXmlConfKeys.KEY_YARN_QUEUE + "=noqueue",
-               ARG_PROVIDER, PROVIDER_HBASE
-           ],
-           true,
-           false,
-           [:])
-      SliderClient sliderClient = (SliderClient) launcher.service
+      ServiceLauncher<SliderClient> launcher =
+          createMasterlessAMWithArgs(clustername,
+              [
+                  Arguments.ARG_COMP_OPT,
+                  SliderKeys.COMPONENT_AM,
+                  RoleKeys.JVM_HEAP, "invalid",
+              ],
+              true,
+              false)
+      SliderClient sliderClient = launcher.service
       addToTeardown(sliderClient);
 
       ApplicationReport report = waitForClusterLive(sliderClient)

Added: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestBadYarnQueue.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestBadYarnQueue.groovy?rev=1594585&view=auto
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestBadYarnQueue.groovy (added)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestBadYarnQueue.groovy Wed May 14 14:19:38 2014
@@ -0,0 +1,75 @@
+/*
+ * 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.agent.standalone
+
+import groovy.transform.CompileStatic
+import groovy.util.logging.Slf4j
+import org.apache.hadoop.yarn.api.records.ApplicationReport
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
+import org.apache.slider.agent.AgentMiniClusterTestBase
+import org.apache.slider.client.SliderClient
+import org.apache.slider.common.SliderExitCodes
+import org.apache.slider.common.SliderXmlConfKeys
+import org.apache.slider.common.params.Arguments
+import org.apache.slider.core.main.ServiceLaunchException
+import org.apache.slider.core.main.ServiceLauncher
+import org.junit.Test
+
+@CompileStatic
+@Slf4j
+
+class TestBadYarnQueue extends AgentMiniClusterTestBase {
+
+  /**
+   * Test disabled because YARN queues don't get validated in the mini cluster
+   * @throws Throwable
+   */
+
+  @Test
+  public void testBadYarnQueue() throws Throwable {
+    skip("untestable in minicluster")
+    String clustername = "test_bad_yarn_queue"
+    createMiniCluster(clustername, configuration, 1, true)
+
+    describe "verify that a bad yarn queue fails the launch"
+
+    try {
+      ServiceLauncher<SliderClient> launcher =
+          createMasterlessAMWithArgs(clustername,
+              [
+                  Arguments.ARG_DEFINE,
+                  SliderXmlConfKeys.KEY_YARN_QUEUE + "=noqueue",
+
+              ],
+              true,
+              false)
+
+      SliderClient sliderClient = launcher.service
+      addToTeardown(sliderClient);
+
+      ApplicationReport report = waitForClusterLive(sliderClient)
+      assert report.yarnApplicationState == YarnApplicationState.FAILED
+      
+    } catch (ServiceLaunchException e) {
+      assertExceptionDetails(e, SliderExitCodes.EXIT_YARN_SERVICE_FAILED)
+    }
+    
+  }
+
+}

Copied: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestBuildStandaloneAM.groovy (from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/build/TestBuildClusterM1W5.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestBuildStandaloneAM.groovy?p2=incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestBuildStandaloneAM.groovy&p1=incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/build/TestBuildClusterM1W5.groovy&r1=1594525&r2=1594585&rev=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/build/TestBuildClusterM1W5.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestBuildStandaloneAM.groovy Wed May 14 14:19:38 2014
@@ -16,50 +16,41 @@
  *  limitations under the License.
  */
 
-package org.apache.slider.providers.hbase.minicluster.build
+package org.apache.slider.agent.standalone
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.slider.core.main.LauncherExitCodes
+import org.apache.hadoop.yarn.api.records.ApplicationReport
+import org.apache.slider.agent.AgentMiniClusterTestBase
+import org.apache.slider.client.SliderClient
 import org.apache.slider.common.SliderExitCodes
-import org.apache.slider.core.exceptions.SliderException
-import org.apache.slider.providers.hbase.HBaseKeys
 import org.apache.slider.common.params.SliderActions
-import org.apache.slider.client.SliderClient
-import org.apache.slider.providers.hbase.minicluster.HBaseMiniClusterTestBase
-import org.apache.hadoop.yarn.api.records.ApplicationReport
+import org.apache.slider.core.exceptions.SliderException
+import org.apache.slider.core.main.LauncherExitCodes
 import org.apache.slider.core.main.ServiceLauncher
 import org.junit.Test
 
-import static HBaseKeys.PROVIDER_HBASE
-import static org.apache.slider.common.params.Arguments.ARG_PROVIDER
-
 @CompileStatic
 @Slf4j
 
-class TestBuildClusterM1W5 extends HBaseMiniClusterTestBase {
+class TestBuildStandaloneAM extends AgentMiniClusterTestBase {
 
   @Test
   public void testBuildCluster() throws Throwable {
-    String clustername = "test_build_cluster_m1_w5"
-    createMiniCluster(clustername, getConfiguration(), 1, true)
+    String clustername = "test_build_cluster"
+    createMiniCluster(clustername, configuration, 1, true)
 
     describe "verify that a build cluster is created but not started"
 
-    ServiceLauncher launcher = createOrBuildCluster(
+    ServiceLauncher<SliderClient> launcher = createOrBuildCluster(
         SliderActions.ACTION_BUILD,
         clustername,
-        [
-            (HBaseKeys.ROLE_MASTER): 1,
-            (HBaseKeys.ROLE_WORKER): 5,
-        ],
-        [
-            ARG_PROVIDER, PROVIDER_HBASE
-        ],
+        [:],
+        [],
         true,
         false,
-        [:])
-    SliderClient sliderClient = (SliderClient) launcher.service
+        agentDefOptions)
+    SliderClient sliderClient = launcher.service
     addToTeardown(sliderClient);
 
     //verify that exists(live) is now false
@@ -75,22 +66,24 @@ class TestBuildClusterM1W5 extends HBase
 
     //and a second attempt will fail as the cluster now exists
     try {
-      createOrBuildCluster(
+      ServiceLauncher<SliderClient> cluster2 = createOrBuildCluster(
           SliderActions.ACTION_BUILD,
           clustername,
-          [
-              (HBaseKeys.ROLE_MASTER): 1,
-              (HBaseKeys.ROLE_WORKER): 3,
-          ],
-          [
-              ARG_PROVIDER, PROVIDER_HBASE
-          ],
-          true,
+          [:],
+          [],
           false,
-          [:])
+          false,
+          agentDefOptions)
+      fail("expected an exception, got $cluster2.service")
     } catch (SliderException e) {
-      assert e.exitCode == SliderExitCodes.EXIT_INSTANCE_EXISTS
+      assertExceptionDetails(e, SliderExitCodes.EXIT_INSTANCE_EXISTS, "")
     }
+
+    //thaw time
+    ServiceLauncher<SliderClient> l2 = thawCluster(clustername, [], true)
+    SliderClient thawed = l2.service
+    addToTeardown(thawed);
+    waitForClusterLive(thawed)
   }
 
 }

Copied: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestDestroyMasterlessAM.groovy (from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestDestroyMasterlessAM.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestDestroyMasterlessAM.groovy?p2=incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestDestroyMasterlessAM.groovy&p1=incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestDestroyMasterlessAM.groovy&r1=1594525&r2=1594585&rev=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestDestroyMasterlessAM.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestDestroyMasterlessAM.groovy Wed May 14 14:19:38 2014
@@ -16,43 +16,43 @@
  *  limitations under the License.
  */
 
-package org.apache.slider.providers.hbase.minicluster.masterless
+package org.apache.slider.agent.standalone
 
+import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
+import org.apache.slider.agent.AgentMiniClusterTestBase
+import org.apache.slider.client.SliderClient
 import org.apache.slider.common.SliderExitCodes
+import org.apache.slider.common.params.ActionEchoArgs
+import org.apache.slider.common.params.Arguments
+import org.apache.slider.common.params.SliderActions
+import org.apache.slider.common.tools.SliderFileSystem
 import org.apache.slider.core.exceptions.ErrorStrings
 import org.apache.slider.core.exceptions.SliderException
 import org.apache.slider.core.exceptions.UnknownApplicationInstanceException
-import org.apache.slider.common.tools.SliderFileSystem
-import org.apache.slider.common.params.Arguments
-import org.apache.slider.common.params.ActionEchoArgs
-import org.apache.slider.common.params.CommonArgs
-import org.apache.slider.client.SliderClient
-import org.apache.slider.providers.hbase.minicluster.HBaseMiniClusterTestBase
 import org.apache.slider.core.main.ServiceLauncher
 import org.junit.Test
 
 /**
- * create masterless AMs and work with them. This is faster than
- * bringing up full clusters
+ * destroy a masterless AM
  */
-//@CompileStatic
+@CompileStatic
 @Slf4j
 
-class TestDestroyMasterlessAM extends HBaseMiniClusterTestBase {
+class TestDestroyMasterlessAM extends AgentMiniClusterTestBase {
 
   @Test
   public void testDestroyMasterlessAM() throws Throwable {
     String clustername = "test_destroy_masterless_am"
-    createMiniCluster(clustername, getConfiguration(), 1, true)
+    createMiniCluster(clustername, configuration, 1, false)
 
     describe "create a masterless AM, stop it, try to create" +
              "a second cluster with the same name, destroy it, try a third time"
 
-    ServiceLauncher launcher1 = launchClientAgainstMiniMR(
-        getConfiguration(),
+    ServiceLauncher<SliderClient> launcher1 = launchClientAgainstMiniMR(
+        configuration,
         [
-            CommonArgs.ACTION_DESTROY,
+            SliderActions.ACTION_DESTROY,
             "no-cluster-of-this-name",
             Arguments.ARG_FILESYSTEM, fsDefaultName,
         ])
@@ -60,16 +60,16 @@ class TestDestroyMasterlessAM extends HB
 
 
 
-    ServiceLauncher launcher = createMasterlessAM(clustername, 0, true, true)
-    SliderClient sliderClient = (SliderClient) launcher.service
+    ServiceLauncher<SliderClient> launcher = createMasterlessAM(clustername, 0, true, true)
+    SliderClient sliderClient = launcher.service
     addToTeardown(sliderClient);
 
     SliderFileSystem sliderFileSystem = createSliderFileSystem()
-    def hdfs = sliderFileSystem.fileSystem
+    def fs = sliderFileSystem.fileSystem
     def instanceDir = sliderFileSystem.buildClusterDirPath(clustername)
 
     assertPathExists(
-        hdfs,
+        fs,
         "cluster path not found",
         instanceDir)
 
@@ -104,19 +104,16 @@ class TestDestroyMasterlessAM extends HB
 
     describe "thaw expected to fail"
     //expect thaw to now fail
-    try {
-      launcher = launch(SliderClient,
-                        configuration,
-                        [
-                            CommonArgs.ACTION_THAW,
-                            clustername,
-                            Arguments.ARG_FILESYSTEM, fsDefaultName,
-                            Arguments.ARG_MANAGER, RMAddr,
-                        ])
-      fail("expected an exception")
-    } catch (UnknownApplicationInstanceException e) {
-      //expected
-    }
+    def ex = launchExpectingException(SliderClient,
+        configuration,
+        "",
+        [
+            SliderActions.ACTION_THAW,
+            clustername,
+            Arguments.ARG_FILESYSTEM, fsDefaultName,
+            Arguments.ARG_MANAGER, RMAddr,
+        ])
+    assert ex instanceof UnknownApplicationInstanceException
 
     describe "thaw completed, checking dir is still absent"
     sliderFileSystem.verifyDirectoryNonexistent(instanceDir)

Copied: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestKillMasterlessAM.groovy (from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestKillMasterlessAM.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestKillMasterlessAM.groovy?p2=incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestKillMasterlessAM.groovy&p1=incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestKillMasterlessAM.groovy&r1=1594525&r2=1594585&rev=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestKillMasterlessAM.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestKillMasterlessAM.groovy Wed May 14 14:19:38 2014
@@ -16,70 +16,57 @@
  *  limitations under the License.
  */
 
-package org.apache.slider.providers.hbase.minicluster.masterless
+package org.apache.slider.agent.standalone
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.slider.providers.hbase.HBaseKeys
-import org.apache.slider.client.SliderClient
-import org.apache.slider.providers.hbase.minicluster.HBaseMiniClusterTestBase
 import org.apache.hadoop.yarn.api.records.ApplicationReport
 import org.apache.hadoop.yarn.api.records.YarnApplicationState
+import org.apache.slider.agent.AgentMiniClusterTestBase
+import org.apache.slider.client.SliderClient
+import org.apache.slider.common.SliderXmlConfKeys
+import org.apache.slider.common.params.Arguments
 import org.apache.slider.core.main.ServiceLauncher
 import org.junit.Test
 
-import static org.apache.slider.providers.hbase.HBaseKeys.PROVIDER_HBASE
-import static org.apache.slider.common.params.Arguments.*
-
 /**
- * create masterless AMs and work with them. This is faster than
- * bringing up full clusters
+ * kill a masterless AM and verify it shuts down. This test
+ * also sets the retry count to 1 to stop recreation attempts
  */
 @CompileStatic
 @Slf4j
 
-class TestKillMasterlessAM extends HBaseMiniClusterTestBase {
+class TestKillMasterlessAM extends AgentMiniClusterTestBase {
 
 
   @Test
   public void testKillMasterlessAM() throws Throwable {
     String clustername = "test_kill_masterless_am"
-    createMiniCluster(clustername, getConfiguration(), 1, true)
+    createMiniCluster(clustername, configuration, 1, true)
 
     describe "kill a masterless AM and verify that it shuts down"
-
-    Map<String, Integer> roles = [
-        (HBaseKeys.ROLE_MASTER): 0,
-        (HBaseKeys.ROLE_WORKER): 0,
-    ]
-    ServiceLauncher launcher = createCluster(clustername,
-        roles,
-        [
-/*
-            ARG_COMP_OPT, SliderKeys.COMPONENT_AM,
-            RoleKeys.JVM_OPTS, "-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5005",
-*/
-            ARG_PROVIDER, PROVIDER_HBASE
-        ],
-        true,
-        true,
-        [:])
-    SliderClient sliderClient = (SliderClient) launcher.service
+    ServiceLauncher<SliderClient> launcher =
+        createMasterlessAMWithArgs(clustername,
+          [
+              Arguments.ARG_OPTION, SliderXmlConfKeys.KEY_AM_RESTART_LIMIT, "1"
+          ],
+          true,
+          false)
+    SliderClient sliderClient = launcher.service
     addToTeardown(sliderClient);
-    describe("listing services")
+    ApplicationReport report = waitForClusterLive(sliderClient)
+
+    describe("listing Java processes")
     lsJavaProcesses();
-    describe("killing services")
-    killServiceLaunchers(SIGTERM);
+    describe("killing AM")
+    killAM(SIGTERM);
     waitWhileClusterLive(sliderClient);
     //give yarn some time to notice
-    sleep(2000)
+    sleep(10000)
     describe("final listing")
     lsJavaProcesses();
-    ApplicationReport report = sliderClient.applicationReport
-    assert report.yarnApplicationState == YarnApplicationState.FAILED;
-
-
-
+    report = sliderClient.applicationReport
+    assert YarnApplicationState.FAILED == report.yarnApplicationState;
     clusterActionFreeze(sliderClient, clustername)
   }
 

Modified: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy?rev=1594585&r1=1594584&r2=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy Wed May 14 14:19:38 2014
@@ -31,7 +31,6 @@ import org.apache.slider.core.exceptions
 import org.apache.slider.core.main.ServiceLauncher
 import org.apache.slider.core.registry.info.ServiceInstanceData
 import org.apache.slider.server.services.curator.CuratorServiceInstance
-import org.apache.slider.server.services.curator.RegistryBinderService
 import org.apache.slider.server.services.registry.SliderRegistryService
 import org.junit.Test
 
@@ -47,8 +46,8 @@ class TestStandaloneAgentAM  extends Age
     //launch fake master
     String clustername = "test_standalone_agent_am"
     createMiniCluster(clustername, configuration, 1, true)
-    ServiceLauncher<SliderClient> launcher
-    launcher = createMasterlessAM(clustername, 0, true, false)
+    ServiceLauncher<SliderClient> launcher =
+        createMasterlessAM(clustername, 0, true, false)
     SliderClient client = launcher.service
     addToTeardown(client);
 
@@ -139,8 +138,6 @@ class TestStandaloneAgentAM  extends Age
         clustername)
     assert i2AppID == instance2.applicationId
 
-
-
     describe("attempting to create instance #3")
     //now try to create instance #3, and expect an in-use failure
     try {

Copied: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneBadClusterName.groovy (from r1594525, incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestBadClusterName.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneBadClusterName.groovy?p2=incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneBadClusterName.groovy&p1=incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestBadClusterName.groovy&r1=1594525&r2=1594585&rev=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/masterless/TestBadClusterName.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneBadClusterName.groovy Wed May 14 14:19:38 2014
@@ -16,47 +16,29 @@
  *  limitations under the License.
  */
 
-package org.apache.slider.providers.hbase.minicluster.masterless
+package org.apache.slider.agent.standalone
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.slider.providers.hbase.HBaseKeys
-import org.apache.slider.client.SliderClient
-import org.apache.slider.providers.hbase.minicluster.HBaseMiniClusterTestBase
+import org.apache.slider.agent.AgentMiniClusterTestBase
 import org.apache.slider.core.main.LauncherExitCodes
 import org.apache.slider.core.main.ServiceLaunchException
-import org.apache.slider.core.main.ServiceLauncher
 import org.junit.Test
 
-import static HBaseKeys.PROVIDER_HBASE
-import static org.apache.slider.common.params.Arguments.ARG_PROVIDER
-
 @CompileStatic
 @Slf4j
 
-class TestBadClusterName extends HBaseMiniClusterTestBase {
+class TestStandaloneBadClusterName extends AgentMiniClusterTestBase {
 
   @Test
-  public void testBadClusterName() throws Throwable {
-    String clustername = "TestBadClusterName"
-    createMiniCluster(clustername, getConfiguration(), 1, true)
+  public void testStandaloneBadClusterName() throws Throwable {
+    String clustername = "TestStandaloneBadClusterName"
+    createMiniCluster(clustername, configuration, 1, true)
 
-    describe "verify that bad cluster are picked up"
+    describe "verify that bad cluster names are picked up"
 
     try {
-      ServiceLauncher launcher = createCluster(clustername,
-           [
-               (HBaseKeys.ROLE_MASTER): 0,
-               (HBaseKeys.ROLE_WORKER): 0,
-           ],
-           [
-               ARG_PROVIDER, PROVIDER_HBASE
-           ],
-           true,
-           false,
-           [:])
-      SliderClient sliderClient = (SliderClient) launcher.service
-      addToTeardown(sliderClient);
+      addToTeardown(createMasterlessAM(clustername, 0, true, false).service);
       fail("expected a failure")
     } catch (ServiceLaunchException e) {
       assertExceptionDetails(e, LauncherExitCodes.EXIT_COMMAND_ARGUMENT_ERROR)

Modified: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy?rev=1594585&r1=1594584&r2=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy Wed May 14 14:19:38 2014
@@ -38,7 +38,6 @@ import org.apache.slider.core.registry.r
 import org.apache.slider.server.appmaster.PublishedArtifacts
 import org.apache.slider.server.appmaster.web.rest.RestPaths
 import org.apache.slider.server.services.curator.CuratorServiceInstance
-import org.apache.slider.server.services.curator.RegistryBinderService
 import org.apache.slider.server.services.registry.SliderRegistryService
 import org.junit.Test
 

Modified: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy?rev=1594585&r1=1594584&r2=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy Wed May 14 14:19:38 2014
@@ -483,21 +483,22 @@ class SliderTestUtils extends Assert {
     return serviceLauncher;
   }
 
-  public static void launchExpectingException(Class serviceClass,
+  public static Throwable launchExpectingException(Class serviceClass,
                                               Configuration conf,
                                               String expectedText,
                                               List args)
       throws Throwable {
     try {
       ServiceLauncher launch = launch(serviceClass, conf, args);
-      fail("Expected an exception with text containing " + expectedText
+      throw new AssertionError("Expected an exception with text containing " + expectedText
                + " -but the service completed with exit code "
                + launch.serviceExitCode);
     } catch (Throwable thrown) {
-      if (!thrown.toString().contains(expectedText)) {
+      if (expectedText && !thrown.toString().contains(expectedText)) {
         //not the right exception -rethrow
         throw thrown;
       }
+      return thrown;
     }
   }
 

Modified: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy?rev=1594585&r1=1594584&r2=1594585&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy Wed May 14 14:19:38 2014
@@ -331,10 +331,7 @@ public abstract class YarnMiniClusterTes
   }
 
 
-  public void killServiceLaunchers(int value) {
-    killAM(value)
-  }
-
+  
   public YarnConfiguration getTestConfiguration() {
     YarnConfiguration conf = getConfiguration()