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 2015/01/12 15:15:09 UTC

incubator-slider git commit: SLIDER-748 TestAgentAMManagementWS.testAgentAMManagementWS failing

Repository: incubator-slider
Updated Branches:
  refs/heads/develop a8de73f04 -> 4c9268b9a


SLIDER-748 TestAgentAMManagementWS.testAgentAMManagementWS failing


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

Branch: refs/heads/develop
Commit: 4c9268b9ae71dad0437262d9019390c9368e3c17
Parents: a8de73f
Author: Steve Loughran <st...@apache.org>
Authored: Mon Jan 12 14:14:46 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Jan 12 14:14:46 2015 +0000

----------------------------------------------------------------------
 .../funtest/accumulo/AccumuloBasicIT.groovy     |   2 +-
 .../server/appmaster/SliderAppMaster.java       |  50 ++--
 .../actions/ActionRegisterServiceInstance.java  |  59 +++++
 .../agent/TestAgentAMManagementWS.groovy        | 253 ++++++++++---------
 .../apache/slider/test/SliderTestUtils.groovy   |  23 +-
 5 files changed, 237 insertions(+), 150 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4c9268b9/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloBasicIT.groovy
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloBasicIT.groovy b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloBasicIT.groovy
index 5f16d17..4f6d6da 100644
--- a/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloBasicIT.groovy
+++ b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloBasicIT.groovy
@@ -205,7 +205,7 @@ class AccumuloBasicIT extends AccumuloAgentCommandTestBase {
   }
 
   public static void checkMonitorPage(String monitorUrl) {
-    String monitor = fetchWebPageWithoutError(monitorUrl);
+    String monitor = fetchWebPageRaisedErrorCodes(monitorUrl);
     assert monitor != null, "Monitor page null"
     assert monitor.length() > 100, "Monitor page too short"
     assert monitor.contains("Accumulo Overview"), "Monitor page didn't contain expected text"

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4c9268b9/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 a6974b9..06e6c1d 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
@@ -116,6 +116,7 @@ import org.apache.slider.providers.agent.AgentKeys;
 import org.apache.slider.providers.slideram.SliderAMClientProvider;
 import org.apache.slider.providers.slideram.SliderAMProviderService;
 import org.apache.slider.server.appmaster.actions.ActionKillContainer;
+import org.apache.slider.server.appmaster.actions.ActionRegisterServiceInstance;
 import org.apache.slider.server.appmaster.actions.RegisterComponentInstance;
 import org.apache.slider.server.appmaster.actions.QueueExecutor;
 import org.apache.slider.server.appmaster.actions.ActionHalt;
@@ -892,9 +893,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
 
     try {
-      // start handling any scheduled events
-
-      startQueueProcessing();
 
       // Web service endpoints: initialize
 
@@ -909,10 +907,10 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
       // start the agent web app
       startAgentWebApp(appInformation, serviceConf, webAppApi);
-      deployWebApplication(serviceConf, webAppPort, webAppApi);
+      deployWebApplication(webAppPort, webAppApi);
 
-      // YARN Registry do the registration
-      registerServiceInstance(clustername, appid);
+      // schedule YARN Registry registration
+      queue(new ActionRegisterServiceInstance(clustername, appid));
 
       // log the YARN and web UIs
       log.info("RM Webapp address {}",
@@ -926,6 +924,10 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       // starts the node review process
       launchProviderService(instanceDefinition, confDir);
 
+      // start handling any scheduled events
+
+      startQueueProcessing();
+
       //now block waiting to be told to exit the process
       waitForAMCompletionSignal();
     } catch(Exception e) {
@@ -942,19 +944,19 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
    *   Creates and starts the web application, and adds a
    *   <code>WebAppService</code> service under the AM, to ensure
    *   a managed web application shutdown.
-   *  @param serviceConf AM configuration
+   * @param serviceConf AM configuration
    * @param port port to deploy the web application on
    * @param webAppApi web app API instance
    */
-  private void deployWebApplication(Configuration serviceConf,
-      int port, WebAppApiImpl webAppApi) {
+  private void deployWebApplication(int port, WebAppApiImpl webAppApi) {
 
+    log.info("Creating and launching web application");
     webApp = new SliderAMWebApp(webAppApi);
     WebApps.$for(SliderAMWebApp.BASE_PATH,
         WebAppApi.class,
         webAppApi,
         RestPaths.WS_CONTEXT)
-           .withHttpPolicy(serviceConf, HttpConfig.Policy.HTTP_ONLY)
+           .withHttpPolicy(getConfig(), HttpConfig.Policy.HTTP_ONLY)
            .at(port)
            .inDevMode()
            .start(webApp);
@@ -962,9 +964,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     WebAppService<SliderAMWebApp> webAppService =
       new WebAppService<SliderAMWebApp>("slider", webApp);
 
-    webAppService.init(serviceConf);
-    webAppService.start();
-    addService(webAppService);
+    deployChildService(webAppService);
   }
 
   private void processAMCredentials(SecurityConfiguration securityConfiguration)
@@ -1148,11 +1148,11 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   /**
    * This registers the service instance and its external values
    * @param instanceName name of this instance
-   * @param appid application ID
+   * @param appId application ID
    * @throws IOException
    */
-  private void registerServiceInstance(String instanceName,
-      ApplicationId appid) throws IOException {
+  public void registerServiceInstance(String instanceName,
+      ApplicationId appId) throws IOException {
     
     
     // the registry is running, so register services
@@ -1173,7 +1173,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
     // Yarn registry
     ServiceRecord serviceRecord = new ServiceRecord();
-    serviceRecord.set(YarnRegistryAttributes.YARN_ID, appid.toString());
+    serviceRecord.set(YarnRegistryAttributes.YARN_ID, appId.toString());
     serviceRecord.set(YarnRegistryAttributes.YARN_PERSISTENCE,
         PersistencePolicies.APPLICATION);
     serviceRecord.description = "Slider Application Master";
@@ -1573,7 +1573,8 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       //  known nodes trigger notifications
       if(!result.unknownNode) {
         getProviderService().notifyContainerCompleted(containerId);
-        queue(new UnregisterComponentInstance(containerId, 0, TimeUnit.MILLISECONDS));
+        queue(new UnregisterComponentInstance(containerId, 0,
+            TimeUnit.MILLISECONDS));
       }
     }
 
@@ -1751,7 +1752,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   public ProtocolSignature getProtocolSignature(String protocol,
                                                 long clientVersion,
                                                 int clientMethodsHash) throws
-                                                                       IOException {
+      IOException {
     return ProtocolSignature.getProtocolSignature(
       this, protocol, clientVersion, clientMethodsHash);
   }
@@ -1882,7 +1883,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     RoleInstance instance = appState.getLiveInstanceByContainerID(
         request.getUuid());
     return Messages.GetNodeResponseProto.newBuilder()
-                   .setClusterNode(instance.toProtobuf())
+                                        .setClusterNode(instance.toProtobuf())
                    .build();
   }
 
@@ -1983,7 +1984,8 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
                                                     File confDir)
     throws IOException, SliderException {
     Map<String, String> env = new HashMap<String, String>();
-    boolean execStarted = providerService.exec(instanceDefinition, confDir, env, this);
+    boolean execStarted = providerService.exec(instanceDefinition, confDir, env,
+        this);
     if (execStarted) {
       providerService.registerServiceListener(this);
       providerService.start();
@@ -2150,7 +2152,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
   @Override //  NMClientAsync.CallbackHandler 
   public void onContainerStarted(ContainerId containerId,
-                                 Map<String, ByteBuffer> allServiceResponse) {
+      Map<String, ByteBuffer> allServiceResponse) {
     LOG_YARN.info("Started Container {} ", containerId);
     RoleInstance cinfo = appState.onNodeManagerContainerStarted(containerId);
     if (cinfo != null) {
@@ -2180,14 +2182,14 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
   @Override //  NMClientAsync.CallbackHandler 
   public void onContainerStatusReceived(ContainerId containerId,
-                                        ContainerStatus containerStatus) {
+      ContainerStatus containerStatus) {
     LOG_YARN.debug("Container Status: id={}, status={}", containerId,
         containerStatus);
   }
 
   @Override //  NMClientAsync.CallbackHandler 
   public void onGetContainerStatusError(
-    ContainerId containerId, Throwable t) {
+      ContainerId containerId, Throwable t) {
     LOG_YARN.error("Failed to query the status of Container {}", containerId);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4c9268b9/slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionRegisterServiceInstance.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionRegisterServiceInstance.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionRegisterServiceInstance.java
new file mode 100644
index 0000000..ca330af
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionRegisterServiceInstance.java
@@ -0,0 +1,59 @@
+/*
+ * 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.server.appmaster.actions;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Asynchronous registration operation
+ */
+public class ActionRegisterServiceInstance extends AsyncAction {
+
+  private final String instanceName;
+  private final ApplicationId appId;
+
+  public ActionRegisterServiceInstance(String instanceName,
+      ApplicationId appId) {
+    super("ActionRegisterServiceInstance");
+    this.instanceName = instanceName;
+    this.appId = appId;
+  }
+
+  public ActionRegisterServiceInstance(String instanceName,
+      ApplicationId appId,
+      long delay,
+      TimeUnit timeUnit) {
+    super("ActionRegisterServiceInstance", delay, timeUnit);
+    this.instanceName = instanceName;
+    this.appId = appId;
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState) throws Exception {
+
+    // YARN Registry do the registration
+    appMaster.registerServiceInstance(instanceName, appId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4c9268b9/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAMManagementWS.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAMManagementWS.groovy b/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAMManagementWS.groovy
index 2e2d6bf..2e2e62b 100644
--- a/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAMManagementWS.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAMManagementWS.groovy
@@ -23,12 +23,11 @@ import com.sun.jersey.api.client.WebResource
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.fs.Path
-import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.exceptions.YarnException
 import org.apache.slider.api.StatusKeys
 import org.apache.slider.client.SliderClient
 import org.apache.slider.common.SliderKeys
-import org.apache.slider.common.SliderXmlConfKeys
+import static org.apache.slider.common.SliderXmlConfKeys.KEY_KEYSTORE_LOCATION
 import org.apache.slider.core.build.InstanceBuilder
 import org.apache.slider.core.conf.AggregateConf
 import org.apache.slider.core.conf.MapOperations
@@ -61,52 +60,61 @@ class TestAgentAMManagementWS extends AgentTestBase {
   private static String password;
 
   public static final String AGENT_URI = "ws/v1/slider/agents/";
-    final static Logger logger = LoggerFactory.getLogger(TestAgentAMManagementWS.class)
-    static {
-        //for localhost testing only
-        HttpsURLConnection.setDefaultHostnameVerifier(
-                new HostnameVerifier(){
-                    public boolean verify(String hostname,
-                                          SSLSession sslSession) {
-                        logger.info("verifying hostname ${hostname}")
-                        InetAddress[] addresses =
-                            InetAddress.getAllByName(hostname);
-                        if (hostname.equals("localhost")) {
-                            return true;
-                        }
-                        for (InetAddress address : addresses) {
-                            if (address.getHostName().equals(hostname) ||
-                                address.isAnyLocalAddress() ||
-                                address.isLoopbackAddress()) {
-                                return true;
-                            }
-                        }
-                        return false;
-                    }
-                });
+  final static Logger logger = LoggerFactory.getLogger(
+      TestAgentAMManagementWS.class)
+  static private String keystoreLocation
+  public static
+  final String SSL_SERVER_KEYSTORE_LOCATION = "ssl.server.keystore.location"
+  public static
+  final String SSL_SERVER_KEYSTORE_PASSWORD = "ssl.server.keystore.password"
+  static {
+    //for localhost testing only
+    HttpsURLConnection.setDefaultHostnameVerifier(
+        new HostnameVerifier() {
+          public boolean verify(String hostname,
+              SSLSession sslSession) {
+            logger.info("verifying hostname ${hostname}")
+            InetAddress[] addresses =
+                InetAddress.getAllByName(hostname);
+            if (hostname.equals("localhost")) {
+              return true;
+            }
+            for (InetAddress address : addresses) {
+              if (address.hostName == hostname ||
+                  address.anyLocalAddress ||
+                  address.loopbackAddress) {
+                return true;
+              }
+            }
+            return false;
+          }
+        });
 
-    }
+  }
 
-    @Override
-    @Before
-    void setup() {
-        super.setup()
-        MapOperations compOperations = new MapOperations();
-        compOperations.put(SliderXmlConfKeys.KEY_KEYSTORE_LOCATION, "/tmp/work/security/keystore.p12");
-        SecurityUtils.initializeSecurityParameters(compOperations, true);
-        CertificateManager certificateManager = new CertificateManager();
-        certificateManager.initialize(compOperations);
-        String keystoreFile = SecurityUtils.getSecurityDir() + File.separator + SliderKeys.KEYSTORE_FILE_NAME;
-        password = SecurityUtils.getKeystorePass();
-        System.setProperty("javax.net.ssl.trustStore", keystoreFile);
-        System.setProperty("javax.net.ssl.trustStorePassword", password);
-        System.setProperty("javax.net.ssl.trustStoreType", "PKCS12");
+  @Override
+  @Before
+  void setup() {
+    super.setup()
+    MapOperations compOperations = new MapOperations();
+    // set keystore demp file
+    keystoreLocation = "/tmp/work/security/keystore.p12" 
+    compOperations[KEY_KEYSTORE_LOCATION] = keystoreLocation
+    SecurityUtils.initializeSecurityParameters(compOperations, true);
+    CertificateManager certificateManager = new CertificateManager();
+    certificateManager.initialize(compOperations);
+    String keystoreFile = SecurityUtils.getSecurityDir() + File.separator +
+                          SliderKeys.KEYSTORE_FILE_NAME;
+    password = SecurityUtils.getKeystorePass();
+    System.setProperty("javax.net.ssl.trustStore", keystoreFile);
+    System.setProperty("javax.net.ssl.trustStorePassword", password);
+    System.setProperty("javax.net.ssl.trustStoreType", "PKCS12");
 
-    }
+  }
 
-    @Test
+  @Test
   public void testAgentAMManagementWS() throws Throwable {
-      String clustername = createMiniCluster("",
+    String clustername = createMiniCluster("",
         configuration,
         1,
         1,
@@ -124,87 +132,98 @@ class TestAgentAMManagementWS extends AgentTestBase {
     assert agt_ver_path.exists()
     assert agt_conf_path.exists()
     try {
-        sliderClientClassName = TestSliderClient.name
-        ServiceLauncher<SliderClient> launcher = buildAgentCluster(clustername,
-            roles,
-            [
-                ARG_OPTION, PACKAGE_PATH, slider_core.absolutePath,
-                ARG_OPTION, APP_DEF, toURIArg(app_def_path),
-                ARG_OPTION, AGENT_CONF, toURIArg(agt_conf_path),
-                ARG_OPTION, AGENT_VERSION, toURIArg(agt_ver_path),
-            ],
-            true, true,
-            true)
-        SliderClient sliderClient = launcher.service
-        def report = waitForClusterLive(sliderClient)
-        def trackingUrl = report.trackingUrl
-        log.info("tracking URL is $trackingUrl")
-        def agent_url = trackingUrl + AGENT_URI
-
-
-        def status = dumpClusterStatus(sliderClient, "agent AM")
-        def liveURL = status.getInfo(StatusKeys.INFO_AM_AGENT_OPS_URL)
-        if (liveURL) {
-          agent_url = liveURL + AGENT_URI
-        }
-
-        log.info("Agent  is $agent_url")
-        log.info("stacks is ${liveURL}stacks")
-        log.info("conf   is ${liveURL}conf")
-
-        execHttpRequest(WEB_STARTUP_TIME) {
-          GET(agent_url)
-        }
-
-        String page = fetchWebPageWithoutError(agent_url);
-        log.info(page);
-
-        //WS get
-        Client client = createTestClient();
-
-
-        WebResource webResource = client.resource(agent_url + "test/register");
-        RegistrationResponse response = webResource.type(MediaType.APPLICATION_JSON)
-                              .post(
-            RegistrationResponse.class,
-            createDummyJSONRegister());
-
-        //TODO: assert failure as actual agent is not started. This test only starts the AM.
-        assert RegistrationStatus.FAILED == response.getResponseStatus();
+      sliderClientClassName = TestSliderClient.name
+      ServiceLauncher<SliderClient> launcher = buildAgentCluster(clustername,
+          roles,
+          [
+              ARG_OPTION, PACKAGE_PATH, slider_core.absolutePath,
+              ARG_OPTION, APP_DEF, toURIArg(app_def_path),
+              ARG_OPTION, AGENT_CONF, toURIArg(agt_conf_path),
+              ARG_OPTION, AGENT_VERSION, toURIArg(agt_ver_path),
+          ],
+          true, true,
+          true)
+      SliderClient sliderClient = launcher.service
+      def report = waitForClusterLive(sliderClient)
+      //def appmaster = report.trackingUrl
+
+
+      def proxyAM = report.trackingUrl
+
+      log.info("tracking URL is $proxyAM")
+
+      // spin awaiting the agent web page coming up.
+      execHttpRequest(WEB_STARTUP_TIME) {
+        GET(proxyAM)
+      }
+
+      def agent_url = proxyAM + AGENT_URI
+
+
+      def status = dumpClusterStatus(sliderClient, "agent AM")
+      def liveURL = status.getInfo(StatusKeys.INFO_AM_AGENT_OPS_URL)
+      if (liveURL) {
+        agent_url = liveURL + AGENT_URI
+      }
+
+      log.info("Agent  is $agent_url")
+      log.info("stacks is ${liveURL}stacks")
+      log.info("conf   is ${liveURL}conf")
+
+      log.info "AM live, now fetching agent at $agent_url"
+      
+      // spin awaiting the agent web page coming up.
+      execHttpRequest(WEB_STARTUP_TIME) {
+        GET(agent_url)
+      }
+
+      String page = fetchWebPageRaisedErrorCodes(agent_url);
+      log.info(page);
+
+      //WS get
+      Client client = createTestClient();
+
+
+      WebResource webResource = client.resource(agent_url + "test/register");
+      RegistrationResponse response = webResource.type(MediaType.APPLICATION_JSON)
+                                      .post(RegistrationResponse.class,
+                                        createDummyJSONRegister());
+
+      // assert failure as actual agent is not started. This test only starts the AM.
+      assert RegistrationStatus.FAILED == response.responseStatus;
     } finally {
       sliderClientClassName = DEFAULT_SLIDER_CLIENT
     }
-    
+
   }
 
   static class TestSliderClient extends SliderClient {
-      @Override
-      protected void persistInstanceDefinition(boolean overwrite,
-                                               Path appconfdir,
-                                               InstanceBuilder builder)
-      throws IOException, SliderException, LockAcquireFailedException {
-          AggregateConf conf = builder.getInstanceDescription()
-          MapOperations component = conf.getAppConfOperations().getComponent("slider-appmaster")
-          component.put(
-                  "ssl.server.keystore.location",
-                  "/tmp/work/security/keystore.p12")
-          component.put("ssl.server.keystore.password", password)
-          super.persistInstanceDefinition(overwrite, appconfdir, builder)
-      }
+    @Override
+    protected void persistInstanceDefinition(boolean overwrite,
+        Path appconfdir,
+        InstanceBuilder builder)
+    throws IOException, SliderException, LockAcquireFailedException {
+      AggregateConf conf = builder.getInstanceDescription()
+      MapOperations component = conf.getAppConfOperations().getComponent(
+          "slider-appmaster")
+      component[SSL_SERVER_KEYSTORE_LOCATION] =
+          keystoreLocation
+      component[SSL_SERVER_KEYSTORE_PASSWORD] = password
+      super.persistInstanceDefinition(overwrite, appconfdir, builder)
+    }
 
-      @Override
-      LaunchedApplication launchApplication(String clustername,
-                                            Path clusterDirectory,
-                                            AggregateConf instanceDefinition,
-                                            boolean debugAM)
-      throws YarnException, IOException {
-        MapOperations component = instanceDefinition.getAppConfOperations().getComponent("slider-appmaster")
-        component.put(
-                  "ssl.server.keystore.location",
-                  "/tmp/work/security/keystore.p12")
-        component.put("ssl.server.keystore.password", password)
-        return super.launchApplication(clustername, clusterDirectory, instanceDefinition, debugAM)
-      }
+    @Override
+    LaunchedApplication launchApplication(String clustername,
+        Path clusterDirectory,
+        AggregateConf instanceDefinition,
+        boolean debugAM)
+    throws YarnException, IOException {
+      MapOperations component = instanceDefinition.appConfOperations.getComponent(
+          "slider-appmaster")
+      component[SSL_SERVER_KEYSTORE_LOCATION]= keystoreLocation
+      component[SSL_SERVER_KEYSTORE_PASSWORD]= password
+      return super.launchApplication(clustername, clusterDirectory, instanceDefinition, debugAM)
+    }
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4c9268b9/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
index 2257ffa..4891f02 100644
--- a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
@@ -425,7 +425,7 @@ class SliderTestUtils extends Assert {
    */
 
   public static String GET(URL url) {
-    return fetchWebPageWithoutError(url.toString())
+    return fetchWebPageRaisedErrorCodes(url.toString())
   }
 
   public static String GET(URL url, String path) {
@@ -438,7 +438,7 @@ class SliderTestUtils extends Assert {
   }
 
   def static String GET(String s) {
-    return fetchWebPageWithoutError(s)
+    return fetchWebPageRaisedErrorCodes(s)
   }
 
   public static String appendToURL(String base, String path) {
@@ -479,10 +479,13 @@ class SliderTestUtils extends Assert {
   /**
    * Fetches a web page asserting that the response code is between 200 and 400.
    * Will error on 400 and 500 series response codes and let 200 and 300 through. 
-   * @param url
-   * @return
+   * @param url URL to get as string
+   * @return body of response
+   * @throws IOException Network IO problems or exit code >= 400 not specifically handled
+   * @throws NotFoundException 404 received
+   * @throws ForbiddenException 401 received
    */
-  public static String fetchWebPageWithoutError(String url) {
+  public static String fetchWebPageRaisedErrorCodes(String url) {
     assert null != url
 
     log.info("Fetching HTTP content at " + url);
@@ -505,18 +508,22 @@ class SliderTestUtils extends Assert {
 
     def body = get.responseBodyAsString
 
-    updateFaults("GET", url, resultCode, body)
+    uprateFaults("GET", url, resultCode, body)
     return body;
   }
 
   /**
-   *  uprate some faults
+   * Generate exceptions from error codes >= 400. Some are converted
+   * into specific exceptions.
    * @param verb HTTP verb
    * @param url URL
    * @param resultCode result code
    * @param body any body
+   * @throws NotFoundException 404 received
+   * @throws ForbiddenException 401 received
+   * @throws IOException any other exit code
    */
-  public static void updateFaults(
+  public static void uprateFaults(
       String verb,
       String url,
       int resultCode,