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/20 20:27:21 UTC

svn commit: r1596353 [1/2] - in /incubator/slider/trunk: slider-core/src/main/java/org/apache/slider/ slider-core/src/main/java/org/apache/slider/common/tools/ slider-core/src/main/java/org/apache/slider/core/registry/docstore/ slider-core/src/main/jav...

Author: stevel
Date: Tue May 20 18:27:20 2014
New Revision: 1596353

URL: http://svn.apache.org/r1596353
Log:
SLIDER-35 core of provider config publishing; tests on hbase showing things aren't quite there yet...

Added:
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/docstore/UriMap.java   (with props)
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java   (with props)
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/utility/PatternValidator.java   (with props)
    incubator/slider/trunk/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentClusterLifecycle.groovy
      - copied, changed from r1596135, incubator/slider/trunk/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestClusterLifecycle.groovy
Removed:
    incubator/slider/trunk/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestClusterLifecycle.groovy
Modified:
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/Slider.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigSet.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryViewForProviders.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/SliderRegistryService.java
    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/server/appmaster/web/view/TestClusterSpecificationBlock.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
    incubator/slider/trunk/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java
    incubator/slider/trunk/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
    incubator/slider/trunk/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
    incubator/slider/trunk/slider-providers/accumulo/slider-accumulo-provider/src/main/java/org/apache/slider/providers/accumulo/AccumuloProviderService.java
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseKeys.java
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java
    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/TestTwoLiveClusters.groovy

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/Slider.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/Slider.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/Slider.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/Slider.java Tue May 20 18:27:20 2014
@@ -42,7 +42,7 @@ public class Slider extends SliderClient
     //turn the args to a list
     List<String> argsList = Arrays.asList(args);
     //create a new list, as the ArrayList type doesn't push() on an insert
-    List<String> extendedArgs = new ArrayList<String>(argsList);
+    List<String> extendedArgs = new ArrayList<>(argsList);
     //insert the service name
     extendedArgs.add(0, SERVICE_CLASSNAME);
     //now have the service launcher do its work

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java Tue May 20 18:27:20 2014
@@ -1342,11 +1342,25 @@ public final class SliderUtils {
     } else {
       fullpath.append(path);
     }
-
     return fullpath.toString();
   }
 
   /**
+   * Append a list of paths, inserting "/" signs as appropriate
+   * @param base
+   * @param paths
+   * @return
+   */
+  public static String appendToURL(String base, String...paths) {
+    String result = base;
+    for (String path : paths) {
+      result = appendToURL(result, path);
+    }
+    return result;
+  }
+  
+  
+  /**
    * Callable for async/scheduled halt
    */
   public static class DelayedHalt extends TimerTask {

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigSet.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigSet.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigSet.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigSet.java Tue May 20 18:27:20 2014
@@ -19,6 +19,7 @@
 package org.apache.slider.core.registry.docstore;
 
 import org.apache.slider.server.appmaster.web.rest.RestPaths;
+import org.apache.slider.server.services.utility.PatternValidator;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
@@ -27,7 +28,6 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
-import java.util.regex.Pattern;
 
 /**
  * Represents a set of configurations for an application, component, etc.
@@ -37,15 +37,15 @@ import java.util.regex.Pattern;
 @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 public class PublishedConfigSet {
 
-  public static final String VALID_NAME_PATTERN = RestPaths.PUBLISHED_CONFIGURATION_REGEXP;
-  public static final String E_INVALID_NAME =
-      "Invalid configuration name -it must match the pattern " +
-      VALID_NAME_PATTERN;
-  private static final Pattern validNames = Pattern.compile(VALID_NAME_PATTERN);
+  private static final PatternValidator validator = new PatternValidator(
+      RestPaths.PUBLISHED_CONFIGURATION_REGEXP);
   
   public Map<String, PublishedConfiguration> configurations =
       new HashMap<>();
 
+  public PublishedConfigSet() {
+  }
+
   /**
    * Put a name -it will be converted to lower case before insertion.
    * Any existing entry will be overwritten (that includes an entry
@@ -62,14 +62,13 @@ public class PublishedConfigSet {
 
   /**
    * Validate the name -restricting it to the set defined in 
-   * {@link #VALID_NAME_PATTERN}
+   * {@link RestPaths#PUBLISHED_CONFIGURATION_REGEXP}
    * @param name name to validate
    * @throws IllegalArgumentException if not a valid name
    */
   public static void validateName(String name) {
-    if (!validNames.matcher(name).matches()) {
-      throw new IllegalArgumentException(E_INVALID_NAME);
-    }
+    validator.validate(name);
+    
   }
 
   public PublishedConfiguration get(String name) {
@@ -92,8 +91,8 @@ public class PublishedConfigSet {
 
   public PublishedConfigSet shallowCopy() {
     PublishedConfigSet that = new PublishedConfigSet();
-    for (Map.Entry<String, PublishedConfiguration> entry : configurations
-        .entrySet()) {
+    for (Map.Entry<String, PublishedConfiguration> entry :
+        configurations.entrySet()) {
       that.put(entry.getKey(), entry.getValue().shallowCopy());
     }
     return that;

Added: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/docstore/UriMap.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/docstore/UriMap.java?rev=1596353&view=auto
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/docstore/UriMap.java (added)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/docstore/UriMap.java Tue May 20 18:27:20 2014
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry.docstore;
+
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.HashMap;
+import java.util.Map;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class UriMap {
+
+  public Map<String, String> uris = new HashMap<>();
+  
+  @JsonIgnore
+  public void put(String key, String value) {
+    uris.put(key, value);
+  }
+}

Propchange: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/docstore/UriMap.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java Tue May 20 18:27:20 2014
@@ -62,10 +62,11 @@ public abstract class AbstractProviderSe
     ProviderService {
   private static final Logger log =
     LoggerFactory.getLogger(AbstractProviderService.class);
-  protected  AggregateConf instanceDefinition;
-  protected StateAccessForProviders stateAccessor;
+  protected StateAccessForProviders amState;
   protected AgentRestOperations restOps;
   protected RegistryViewForProviders registry;
+  protected ServiceInstanceData registryInstanceData;
+  protected URL amWebAPI;
 
   public AbstractProviderService(String name) {
     super(name);
@@ -76,18 +77,18 @@ public abstract class AbstractProviderSe
     return getConfig();
   }
 
-  public StateAccessForProviders getStateAccessor() {
-    return stateAccessor;
+  public StateAccessForProviders getAmState() {
+    return amState;
   }
 
-  public void setStateAccessor(StateAccessForProviders stateAccessor) {
-    this.stateAccessor = stateAccessor;
+  public void setAmState(StateAccessForProviders amState) {
+    this.amState = amState;
   }
 
   @Override
   public void bind(StateAccessForProviders stateAccessor,
       RegistryViewForProviders registry) {
-    this.stateAccessor = stateAccessor;
+    this.amState = stateAccessor;
     this.registry = registry;
   }
 
@@ -308,6 +309,7 @@ public abstract class AbstractProviderSe
       ServiceInstanceData registryInstanceData) throws MalformedURLException,
       IOException {
 
-      //no-op
+      this.amWebAPI = amWebAPI;
+    this.registryInstanceData = registryInstanceData;
   }
 }

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java Tue May 20 18:27:20 2014
@@ -72,7 +72,6 @@ import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
-import java.net.MalformedURLException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
@@ -171,7 +170,6 @@ public class AgentProviderService extend
       }
     }
 
-    this.instanceDefinition = instanceDefinition;
     log.info("Build launch context for Agent");
     log.debug(instanceDefinition.toString());
 
@@ -262,13 +260,13 @@ public class AgentProviderService extend
     pubconf.description = description;
     pubconf.putValues(entries);
     log.info("publishing {}", pubconf);
-    getStateAccessor().getPublishedConfigurations().put(name, pubconf);
+    getAmState().getPublishedSliderConfigurations().put(name, pubconf);
   }
 
   protected Map<String, Map<String, ClusterNode>> getRoleClusterNodeMapping() {
-    stateAccessor.refreshClusterStatus();
+    amState.refreshClusterStatus();
     return (Map<String, Map<String, ClusterNode>>)
-        stateAccessor.getClusterStatus().status.get(
+        amState.getClusterStatus().status.get(
             ClusterDescriptionKeys.KEY_CLUSTER_LIVE);
   }
 
@@ -277,7 +275,7 @@ public class AgentProviderService extend
   }
 
   protected String getClusterInfoPropertyValue(String name) {
-    StateAccessForProviders accessor = getStateAccessor();
+    StateAccessForProviders accessor = getAmState();
     assert accessor.isApplicationLive();
     ClusterDescription description = accessor.getClusterStatus();
     return description.getInfo(name);
@@ -367,7 +365,7 @@ public class AgentProviderService extend
     String label = heartBeat.getHostname();
     String roleName = getRoleName(label);
     String containerId = getContainerId(label);
-    StateAccessForProviders accessor = getStateAccessor();
+    StateAccessForProviders accessor = getAmState();
     String scriptPath = getScriptPathFromMetainfo(roleName);
 
     if (scriptPath == null) {
@@ -550,10 +548,10 @@ public class AgentProviderService extend
 
   protected void addInstallCommand(String roleName, String containerId, HeartBeatResponse response, String scriptPath)
       throws SliderException {
-    assert getStateAccessor().isApplicationLive();
-    ConfTreeOperations appConf = getStateAccessor().getAppConfSnapshot();
-    ConfTreeOperations resourcesConf = getStateAccessor().getResourcesSnapshot();
-    ConfTreeOperations internalsConf = getStateAccessor().getInternalsSnapshot();
+    assert getAmState().isApplicationLive();
+    ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
+    ConfTreeOperations resourcesConf = getAmState().getResourcesSnapshot();
+    ConfTreeOperations internalsConf = getAmState().getInternalsSnapshot();
 
     ExecutionCommand cmd = new ExecutionCommand(AgentCommandType.EXECUTION_COMMAND);
     prepareExecutionCommand(cmd);
@@ -597,7 +595,7 @@ public class AgentProviderService extend
   }
 
   private void setInstallCommandConfigurations(ExecutionCommand cmd) {
-    ConfTreeOperations appConf = getStateAccessor().getAppConfSnapshot();
+    ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
     Map<String, Map<String, String>> configurations = buildCommandConfigurations(appConf);
     cmd.setConfigurations(configurations);
   }
@@ -605,9 +603,9 @@ public class AgentProviderService extend
   @VisibleForTesting
   protected void addStatusCommand(String roleName, String containerId, HeartBeatResponse response, String scriptPath)
       throws SliderException {
-    assert getStateAccessor().isApplicationLive();
-    ConfTreeOperations appConf = getStateAccessor().getAppConfSnapshot();
-    ConfTreeOperations internalsConf = getStateAccessor().getInternalsSnapshot();
+    assert getAmState().isApplicationLive();
+    ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
+    ConfTreeOperations internalsConf = getAmState().getInternalsSnapshot();
 
     StatusCommand cmd = new StatusCommand();
     String clusterName = internalsConf.get(OptionKeys.APPLICATION_NAME);
@@ -635,8 +633,8 @@ public class AgentProviderService extend
   @VisibleForTesting
   protected void addGetConfigCommand(String roleName, String containerId, HeartBeatResponse response)
       throws SliderException {
-    assert getStateAccessor().isApplicationLive();
-    ConfTreeOperations internalsConf = getStateAccessor().getInternalsSnapshot();
+    assert getAmState().isApplicationLive();
+    ConfTreeOperations internalsConf = getAmState().getInternalsSnapshot();
 
     StatusCommand cmd = new StatusCommand();
     String clusterName = internalsConf.get(OptionKeys.APPLICATION_NAME);
@@ -659,9 +657,9 @@ public class AgentProviderService extend
   protected void addStartCommand(String roleName, String containerId, HeartBeatResponse response, String scriptPath)
       throws
       SliderException {
-    assert getStateAccessor().isApplicationLive();
-    ConfTreeOperations appConf = getStateAccessor().getAppConfSnapshot();
-    ConfTreeOperations internalsConf = getStateAccessor().getInternalsSnapshot();
+    assert getAmState().isApplicationLive();
+    ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
+    ConfTreeOperations internalsConf = getAmState().getInternalsSnapshot();
 
     ExecutionCommand cmd = new ExecutionCommand(AgentCommandType.EXECUTION_COMMAND);
     prepareExecutionCommand(cmd);

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java Tue May 20 18:27:20 2014
@@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.conf.YarnC
 import org.apache.slider.common.SliderKeys;
 import org.apache.slider.common.tools.ConfigHelper;
 import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.core.conf.AggregateConf;
 import org.apache.slider.core.conf.MapOperations;
 import org.apache.slider.core.exceptions.BadCommandArgumentsException;
@@ -42,7 +43,6 @@ import org.apache.slider.providers.Provi
 import org.apache.slider.providers.ProviderRole;
 import org.apache.slider.providers.agent.AgentKeys;
 import org.apache.slider.server.appmaster.PublishedArtifacts;
-import org.apache.slider.server.appmaster.state.StateAccessForProviders;
 import org.apache.slider.server.appmaster.web.rest.RestPaths;
 import org.apache.slider.server.services.utility.EventCallback;
 
@@ -54,7 +54,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.slider.server.appmaster.web.rest.RestPaths.SLIDER_PATH_AGENTS;
 import static org.apache.slider.server.appmaster.web.rest.RestPaths.SLIDER_PATH_MANAGEMENT;
 import static org.apache.slider.server.appmaster.web.rest.RestPaths.SLIDER_PATH_PUBLISHER;
 
@@ -117,34 +116,31 @@ public class SliderAMProviderService ext
 
     // now publish site.xml files
     YarnConfiguration defaultYarnConfig = new YarnConfiguration();
-    stateAccessor.getPublishedConfigurations().put(
+    amState.getPublishedSliderConfigurations().put(
         PublishedArtifacts.COMPLETE_CONFIG,
         new PublishedConfiguration(
             "Complete slider application settings",
             getConfig(), getConfig())
     );
-    stateAccessor.getPublishedConfigurations().put(
+    amState.getPublishedSliderConfigurations().put(
         PublishedArtifacts.YARN_SITE_CONFIG,
         new PublishedConfiguration(
             "YARN site settings",
             ConfigHelper.loadFromResource("yarn-site.xml"),
-            defaultYarnConfig)
-    );
+            defaultYarnConfig) );
 
-    stateAccessor.getPublishedConfigurations().put(
+    amState.getPublishedSliderConfigurations().put(
         PublishedArtifacts.CORE_SITE_CONFIG,
         new PublishedConfiguration(
             "Core site settings",
             ConfigHelper.loadFromResource("core-site.xml"),
-            defaultYarnConfig)
-    );
-    stateAccessor.getPublishedConfigurations().put(
+            defaultYarnConfig) );
+    amState.getPublishedSliderConfigurations().put(
         PublishedArtifacts.HDFS_SITE_CONFIG,
         new PublishedConfiguration(
             "HDFS site settings",
             ConfigHelper.loadFromResource("hdfs-site.xml"),
-            new HdfsConfiguration(true))
-    );
+            new HdfsConfiguration(true)) );
 
 
     try {
@@ -158,34 +154,30 @@ public class SliderAMProviderService ext
           CustomRegistryConstants.MANAGEMENT_REST_API,
           new RegisteredEndpoint(
               new URL(amWebAPI, SLIDER_PATH_MANAGEMENT),
-              "Management REST API")
-      );
+              "Management REST API") );
 
       externalView.endpoints.put(
           CustomRegistryConstants.REGISTRY_REST_API,
           new RegisteredEndpoint(
               new URL(amWebAPI, RestPaths.SLIDER_PATH_REGISTRY + "/" +
                                 RestPaths.REGISTRY_SERVICE),
-              "Registry Web Service"
-          )
-      );
+              "Registry Web Service" ) );
 
       URL publisherURL = new URL(amWebAPI, SLIDER_PATH_PUBLISHER);
       externalView.endpoints.put(
           CustomRegistryConstants.PUBLISHER_REST_API,
           new RegisteredEndpoint(
               publisherURL,
-              "Publisher Service")
-      );
+              "Publisher Service") );
       
     /*
      * Set the configurations URL.
      */
-      externalView.configurationsURL = publisherURL.toExternalForm();
+      externalView.configurationsURL = SliderUtils.appendToURL(
+          publisherURL.toExternalForm(), RestPaths.SLIDER_CONFIGSET);
 
     } catch (URISyntaxException e) {
       throw new IOException(e);
     }
-
   }
 }

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java Tue May 20 18:27:20 2014
@@ -100,6 +100,7 @@ import org.apache.slider.server.appmaste
 import org.apache.slider.server.appmaster.state.AppState;
 import org.apache.slider.server.appmaster.state.ContainerAssignment;
 import org.apache.slider.server.appmaster.state.ContainerReleaseOperation;
+import org.apache.slider.server.appmaster.state.ProviderAppState;
 import org.apache.slider.server.appmaster.state.RMOperationHandler;
 import org.apache.slider.server.appmaster.state.RoleInstance;
 import org.apache.slider.server.appmaster.state.RoleStatus;
@@ -223,6 +224,9 @@ public class SliderAppMaster extends Abs
    */
   private final AppState appState = new AppState(new ProtobufRecordFactory());
 
+  private final ProviderAppState stateForProviders =
+      new ProviderAppState("undefined", appState);
+
 
   /**
    * model the state using locks and conditions
@@ -379,13 +383,16 @@ public class SliderAppMaster extends Abs
     String action = serviceArgs.getAction();
     List<String> actionArgs = serviceArgs.getActionArgs();
     int exitCode;
-    if (action.equals(SliderActions.ACTION_HELP)) {
-      log.info(getName() + serviceArgs.usage());
-      exitCode = LauncherExitCodes.EXIT_USAGE;
-    } else if (action.equals(SliderActions.ACTION_CREATE)) {
-      exitCode = createAndRunCluster(actionArgs.get(0));
-    } else {
-      throw new SliderException("Unimplemented: " + action);
+    switch (action) {
+      case SliderActions.ACTION_HELP:
+        log.info(getName() + serviceArgs.usage());
+        exitCode = LauncherExitCodes.EXIT_USAGE;
+        break;
+      case SliderActions.ACTION_CREATE:
+        exitCode = createAndRunCluster(actionArgs.get(0));
+        break;
+      default:
+        throw new SliderException("Unimplemented: " + action);
     }
     log.info("Exiting AM; final exit code = {}", exitCode);
     return exitCode;
@@ -425,10 +432,13 @@ public class SliderAppMaster extends Abs
 
     AggregateConf instanceDefinition =
       InstanceIO.loadInstanceDefinitionUnresolved(fs, clusterDirPath);
+    instanceDefinition.setName(clustername);
 
     log.info("Deploying cluster {}:", instanceDefinition);
 
-    //REVISIT: why is this done?
+    stateForProviders.setApplicationName(clustername);
+    
+    // triggers resolution and snapshotting in agent
     appState.updateInstanceDefinition(instanceDefinition);
     File confDir = getLocalConfDir();
     if (!confDir.exists() || !confDir.isDirectory()) {
@@ -561,7 +571,7 @@ public class SliderAppMaster extends Abs
       // Start up the WebApp and track the URL for it
       webApp = new SliderAMWebApp(registry);
       WebApps.$for(SliderAMWebApp.BASE_PATH, WebAppApi.class,
-          new WebAppApiImpl(this, appState, providerService),
+          new WebAppApiImpl(this, stateForProviders, providerService),
           RestPaths.WS_CONTEXT)
                       .with(serviceConf)
                       .start(webApp);
@@ -665,8 +675,8 @@ public class SliderAppMaster extends Abs
 
 
     //Give the provider restricted access to the state, registry
-    providerService.bind(appState, registry);
-    sliderAMProvider.bind(appState, registry);
+    providerService.bind(stateForProviders, registry);
+    sliderAMProvider.bind(stateForProviders, registry);
 
     // now do the registration
     registerServiceInstance(clustername, appid);

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java Tue May 20 18:27:20 2014
@@ -88,7 +88,7 @@ import static org.apache.slider.api.Role
  * is not synchronized and intended to be used during
  * initialization.
  */
-public class AppState implements StateAccessForProviders {
+public class AppState {
   protected static final Logger log =
     LoggerFactory.getLogger(AppState.class);
   
@@ -142,10 +142,6 @@ public class AppState implements StateAc
    */
   private ClusterDescription clusterSpec = new ClusterDescription();
 
-  private final PublishedConfigSet
-      publishedConfigurations = new PublishedConfigSet();
-
-
   private final Map<Integer, RoleStatus> roleStatusMap =
     new ConcurrentHashMap<>();
 
@@ -308,7 +304,7 @@ public class AppState implements StateAc
     return completionOfUnknownContainerEvent;
   }
 
-  @Override
+
   public Map<Integer, RoleStatus> getRoleStatusMap() {
     return roleStatusMap;
   }
@@ -326,18 +322,11 @@ public class AppState implements StateAc
   }
 
 
-  @Override
-  public PublishedConfigSet getPublishedConfigurations() {
-    return publishedConfigurations;
-  }  
-  
-
-  @Override
   public Map<ContainerId, RoleInstance> getFailedNodes() {
     return failedNodes;
   }
 
-  @Override
+
   public Map<ContainerId, RoleInstance> getLiveNodes() {
     return liveNodes;
   }
@@ -350,7 +339,7 @@ public class AppState implements StateAc
     return clusterSpec;
   }
 
-  @Override
+
   public ClusterDescription getClusterStatus() {
     return clusterStatus;
   }
@@ -409,33 +398,33 @@ public class AppState implements StateAc
     containerMaxMemory = maxMemory;
   }
 
-  @Override
+
   public ConfTreeOperations getResourcesSnapshot() {
     return resourcesSnapshot;
   }
 
-  @Override
+
   public ConfTreeOperations getAppConfSnapshot() {
     return appConfSnapshot;
   }
 
-  @Override
+
   public ConfTreeOperations getInternalsSnapshot() {
     return internalsSnapshot;
   }
 
-  @Override
+
   public boolean isApplicationLive() {
     return applicationLive;
   }
 
 
-  @Override
+
   public long getSnapshotTime() {
     return snapshotTime;
   }
 
-  @Override
+
   public AggregateConf getInstanceDefinitionSnapshot() {
     return instanceDefinitionSnapshot;
   }
@@ -607,6 +596,7 @@ public class AppState implements StateAc
     instanceDefinitionSnapshot = new AggregateConf(resourcesSnapshot.confTree,
                                                    appConfSnapshot.confTree,
                                                    internalsSnapshot.confTree);
+    instanceDefinitionSnapshot.setName(instanceDefinition.getName());
 
     clusterSpec =
       ClusterDescriptionOperations.buildFromInstanceDefinition(
@@ -751,7 +741,7 @@ public class AppState implements StateAc
     return appMasterNode;
   }
 
-  @Override
+
   public RoleStatus lookupRoleStatus(int key) {
     RoleStatus rs = getRoleStatusMap().get(key);
     if (rs == null) {
@@ -760,13 +750,13 @@ public class AppState implements StateAc
     return rs;
   }
 
-  @Override
+
   public RoleStatus lookupRoleStatus(Container c) throws YarnRuntimeException {
     return lookupRoleStatus(ContainerPriority.extractRole(c));
   }
 
 
-  @Override
+
   public RoleStatus lookupRoleStatus(String name) throws YarnRuntimeException {
     ProviderRole providerRole = roles.get(name);
     if (providerRole == null) {
@@ -775,23 +765,23 @@ public class AppState implements StateAc
     return lookupRoleStatus(providerRole.id);
   }
 
-  @Override
+
   public synchronized List<RoleInstance> cloneActiveContainerList() {
     Collection<RoleInstance> values = activeContainers.values();
     return new ArrayList<>(values);
   }
   
-  @Override
+
   public int getNumActiveContainers() {
     return activeContainers.size();
   }
   
-  @Override
+
   public RoleInstance getActiveContainer(ContainerId id) {
     return activeContainers.get(id);
   }
 
-  @Override
+
   public synchronized List<RoleInstance> cloneLiveContainerInfoList() {
     List<RoleInstance> allRoleInstances;
     Collection<RoleInstance> values = getLiveNodes().values();
@@ -800,7 +790,7 @@ public class AppState implements StateAc
   }
 
 
-  @Override
+
   public synchronized RoleInstance getLiveInstanceByContainerID(String containerId)
     throws NoSuchNodeException {
     Collection<RoleInstance> nodes = getLiveNodes().values();
@@ -813,7 +803,7 @@ public class AppState implements StateAc
     throw new NoSuchNodeException(containerId);
   }
 
-  @Override
+
   public synchronized List<RoleInstance> getLiveInstancesByContainerIDs(
     Collection<String> containerIDs) {
     //first, a hashmap of those containerIDs is built up
@@ -1152,7 +1142,7 @@ public class AppState implements StateAc
     public RoleInstance roleInstance;
     public boolean containerFailed;
 
-    @Override
+  
     public String toString() {
       final StringBuilder sb =
         new StringBuilder("NodeCompletionResult{");
@@ -1311,7 +1301,7 @@ public class AppState implements StateAc
     return percentage;
   }
 
-  @Override
+
   public void refreshClusterStatus() {
     refreshClusterStatus(null);
   }

Added: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java?rev=1596353&view=auto
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java (added)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java Tue May 20 18:27:20 2014
@@ -0,0 +1,202 @@
+/*
+ * 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.state;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.slider.api.ClusterDescription;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.exceptions.NoSuchNodeException;
+import org.apache.slider.core.registry.docstore.PublishedConfigSet;
+import org.apache.slider.server.appmaster.web.rest.RestPaths;
+import org.apache.slider.server.services.utility.PatternValidator;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class ProviderAppState implements StateAccessForProviders {
+
+
+  private final Map<String, PublishedConfigSet> publishedConfigSets =
+      new ConcurrentHashMap<>(5);
+  private static final PatternValidator validator = new PatternValidator(
+      RestPaths.PUBLISHED_CONFIGURATION_SET_REGEXP);
+  private String applicationName;
+
+  private final AppState appState;
+
+  public ProviderAppState(String applicationName, AppState appState) {
+    this.appState = appState;
+    this.applicationName = applicationName;
+  }
+
+  public void setApplicationName(String applicationName) {
+    this.applicationName = applicationName;
+  }
+
+  @Override
+  public String getApplicationName() {
+    return applicationName;
+  }
+
+  @Override
+  public PublishedConfigSet getPublishedSliderConfigurations() {
+    return getOrCreatePublishedConfigSet(RestPaths.SLIDER_CONFIGSET);
+  }
+
+  @Override
+  public PublishedConfigSet getPublishedConfigSet(String name) {
+    return publishedConfigSets.get(name);
+  }
+
+  @Override
+  public PublishedConfigSet getOrCreatePublishedConfigSet(String name) {
+    PublishedConfigSet set = publishedConfigSets.get(name);
+    if (set == null) {
+      validator.validate(name);
+      synchronized (publishedConfigSets) {
+        // synchronized double check to ensure that there is never an overridden
+        // config set created
+        set = publishedConfigSets.get(name);
+        if (set == null) {
+          set = new PublishedConfigSet();
+          publishedConfigSets.put(name, set);
+        }
+      }
+    }
+    return set;
+  }
+
+  @Override
+  public List<String> listConfigSets() {
+
+    synchronized (publishedConfigSets) {
+      List<String> sets = new ArrayList<>(publishedConfigSets.keySet());
+      return sets;
+    }
+  }
+
+  @Override
+  public Map<Integer, RoleStatus> getRoleStatusMap() {
+    return appState.getRoleStatusMap();
+  }
+
+
+  @Override
+  public Map<ContainerId, RoleInstance> getFailedNodes() {
+    return appState.getFailedNodes();
+  }
+
+  @Override
+  public Map<ContainerId, RoleInstance> getLiveNodes() {
+    return appState.getLiveNodes();
+  }
+
+  @Override
+  public ClusterDescription getClusterStatus() {
+    return appState.getClusterStatus();
+  }
+
+  @Override
+  public ConfTreeOperations getResourcesSnapshot() {
+    return appState.getResourcesSnapshot();
+  }
+
+  @Override
+  public ConfTreeOperations getAppConfSnapshot() {
+    return appState.getAppConfSnapshot();
+  }
+
+  @Override
+  public ConfTreeOperations getInternalsSnapshot() {
+    return appState.getInternalsSnapshot();
+  }
+
+  @Override
+  public boolean isApplicationLive() {
+    return appState.isApplicationLive();
+  }
+
+  @Override
+  public long getSnapshotTime() {
+    return appState.getSnapshotTime();
+  }
+
+  @Override
+  public AggregateConf getInstanceDefinitionSnapshot() {
+    return appState.getInstanceDefinitionSnapshot();
+  }
+
+  @Override
+  public RoleStatus lookupRoleStatus(int key) {
+    return appState.lookupRoleStatus(key);
+  }
+
+  @Override
+  public RoleStatus lookupRoleStatus(Container c) throws YarnRuntimeException {
+    return appState.lookupRoleStatus(c);
+  }
+
+  @Override
+  public RoleStatus lookupRoleStatus(String name) throws YarnRuntimeException {
+    return appState.lookupRoleStatus(name);
+  }
+
+  @Override
+  public List<RoleInstance> cloneActiveContainerList() {
+    return appState.cloneActiveContainerList();
+  }
+
+  @Override
+  public int getNumActiveContainers() {
+    return appState.getNumActiveContainers();
+  }
+
+  @Override
+  public RoleInstance getActiveContainer(ContainerId id) {
+    return appState.getActiveContainer(id);
+  }
+
+  @Override
+  public List<RoleInstance> cloneLiveContainerInfoList() {
+    return appState.cloneLiveContainerInfoList();
+  }
+
+  @Override
+  public RoleInstance getLiveInstanceByContainerID(String containerId) throws
+      NoSuchNodeException {
+    return appState.getLiveInstanceByContainerID(containerId);
+  }
+
+  @Override
+  public List<RoleInstance> getLiveInstancesByContainerIDs(Collection<String> containerIDs) {
+    return appState.getLiveInstancesByContainerIDs(containerIDs);
+  }
+
+  @Override
+  public void refreshClusterStatus() {
+    appState.refreshClusterStatus();
+  }
+
+}

Propchange: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java Tue May 20 18:27:20 2014
@@ -35,13 +35,40 @@ import java.util.Map;
  * The methods to offer state access to the providers
  */
 public interface StateAccessForProviders {
+
   Map<Integer, RoleStatus> getRoleStatusMap();
 
   /**
+   * Get the name of the application
+   * @return the name
+   */
+  String getApplicationName();
+
+  /**
    * Get the published configurations
    * @return the configuration set
    */
-  PublishedConfigSet getPublishedConfigurations();
+  PublishedConfigSet getPublishedSliderConfigurations();
+
+  /**
+   * Get a named published config set
+   * @param name name to look up
+   * @return the instance or null
+   */
+  PublishedConfigSet getPublishedConfigSet(String name);
+
+  /**
+   * Get a named published config set, creating it if need be.
+   * @param name name to look up
+   * @return the instance -possibly a new one
+   */
+  PublishedConfigSet getOrCreatePublishedConfigSet(String name);
+
+  /**
+   * List the config sets -this takes a clone of the current set
+   * @return a list of config sets
+   */
+  List<String> listConfigSets();
 
   Map<ContainerId, RoleInstance> getFailedNodes();
 
@@ -163,4 +190,6 @@ public interface StateAccessForProviders
    * @param providerStatus status from the provider for the cluster info section
    */
   void refreshClusterStatus();
+  
+
 }

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java Tue May 20 18:27:20 2014
@@ -50,7 +50,10 @@ public class RestPaths {
    * fragments: {@value}
    */
   public static final String PUBLISHED_CONFIGURATION_REGEXP
-      ="[a-z0-9][a-z0-9_\\+-]*";
-  
-  
+      = "[a-z0-9][a-z0-9_\\+-]*";
+
+  public static final String PUBLISHED_CONFIGURATION_SET_REGEXP
+      = "[a-z0-9][a-z0-9_.\\+-]*";
+
+  public static final String SLIDER_CONFIGSET = "slider";
 }

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java Tue May 20 18:27:20 2014
@@ -23,6 +23,8 @@ import org.apache.slider.core.registry.d
 import org.apache.slider.core.registry.docstore.PublishedConfigSet;
 import org.apache.slider.core.registry.docstore.PublishedConfiguration;
 import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter;
+import org.apache.slider.core.registry.docstore.UriMap;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
 import org.apache.slider.server.appmaster.web.WebAppApi;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -48,11 +50,15 @@ public class PublisherResource {
   protected static final Logger log =
       LoggerFactory.getLogger(PublisherResource.class);
   private final WebAppApi slider;
+  public static final String SET_NAME = 
+      "{setname: " + PUBLISHED_CONFIGURATION_SET_REGEXP + "}";
   private static final String CONFIG =
-      "{config: " + PUBLISHED_CONFIGURATION_REGEXP + "}";
-  
+      SET_NAME + "/{config: " + PUBLISHED_CONFIGURATION_REGEXP + "}";
+  private final StateAccessForProviders appState;
+
   public PublisherResource(WebAppApi slider) {
     this.slider = slider;
+    appState = slider.getAppState();
   }
 
   private void init(HttpServletResponse res, UriInfo uriInfo) {
@@ -60,20 +66,51 @@ public class PublisherResource {
     log.debug(uriInfo.getRequestUri().toString());
   }
 
-  private PublishedConfigSet getContent() {
-    return slider.getAppState().getPublishedConfigurations();
+  /**
+   * Get a named config set 
+   * @param setname name of the config set
+   * @return the config set
+   * @throws NotFoundException if there was no matching set
+   */
+  private PublishedConfigSet getConfigSet(String setname) {
+    PublishedConfigSet configSet =
+        appState.getPublishedConfigSet(setname);
+    if (configSet == null) {
+      throw new NotFoundException("Not found: " + setname);
+    }
+    return configSet;
   }
 
   @GET
   @Path("/")
   @Produces({MediaType.APPLICATION_JSON})
+  public UriMap enumConfigSets(
+      @Context UriInfo uriInfo,
+      @Context HttpServletResponse res) {
+    init(res, uriInfo);
+    String baseURL = uriInfo.getRequestUri().toString();
+    if (!baseURL.endsWith("/")) {
+      baseURL += "/";
+    }
+    UriMap uriMap = new UriMap();
+    for (String name : appState.listConfigSets()) {
+      uriMap.put(name, baseURL + name);
+    }
+    return uriMap;
+  }
+
+  @GET
+  @Path("/"+ SET_NAME)
+  @Produces({MediaType.APPLICATION_JSON})
   public PublishedConfigSet getPublishedConfiguration(
+      @PathParam("setname") String setname,
       @Context UriInfo uriInfo,
       @Context HttpServletResponse res) {
     init(res, uriInfo);
 
-    PublishedConfigSet publishedConfigSet = getContent();
-    log.debug("number of available configurations: {}", publishedConfigSet.size());
+    logRequest(uriInfo);
+    PublishedConfigSet publishedConfigSet = getConfigSet(setname);
+    log.debug("Number of configurations: {}", publishedConfigSet.size());
     return publishedConfigSet.shallowCopy();
   }
 
@@ -85,27 +122,43 @@ public class PublisherResource {
   @Path("/" + CONFIG)
   @Produces({MediaType.APPLICATION_JSON})
   public PublishedConfiguration getConfigurationInstance(
+      @PathParam("setname") String setname,
       @PathParam("config") String config,
       @Context UriInfo uriInfo,
       @Context HttpServletResponse res) {
     init(res, uriInfo);
 
-    PublishedConfiguration publishedConfig = getContent().get(config);
+    PublishedConfiguration publishedConfig =
+        getPublishedConfiguration(setname, config);
     if (publishedConfig == null) {
       log.info("Configuration {} not found", config);
       throw new NotFoundException("Not found: " + uriInfo.getAbsolutePath());
     }
     return publishedConfig;
   }
-  
+
+  /**
+   * Get a configuration
+   * @param setname name of the config set
+   * @param config config
+   * @return null if there was a config, but not a set
+   * @throws NotFoundException if there was no matching set
+   */
+  public PublishedConfiguration getPublishedConfiguration(String setname,
+      String config) {
+    return getConfigSet(setname).get(config);
+  }
+
   @GET
   @Path("/" + CONFIG+ ".json")
   @Produces({MediaType.APPLICATION_JSON})
   public String getConfigurationContentJson(
+      @PathParam("setname") String setname,
+
       @PathParam("config") String config,
       @Context UriInfo uriInfo,
       @Context HttpServletResponse res) throws IOException {
-    return getStringRepresentation(config, uriInfo, res,
+    return getStringRepresentation(setname, config, uriInfo, res,
         ConfigFormat.JSON);
   }
 
@@ -113,10 +166,11 @@ public class PublisherResource {
   @Path("/" + CONFIG + ".xml")
   @Produces({MediaType.APPLICATION_XML})
   public String getConfigurationContentXML(
+      @PathParam("setname") String setname,
       @PathParam("config") String config,
       @Context UriInfo uriInfo,
       @Context HttpServletResponse res) throws IOException {
-    return getStringRepresentation(config, uriInfo, res,
+    return getStringRepresentation(setname, config, uriInfo, res,
         ConfigFormat.XML);
   }
   
@@ -124,39 +178,43 @@ public class PublisherResource {
   @Path("/" + CONFIG + ".properties")
   @Produces({MediaType.APPLICATION_XML})
   public String getConfigurationContentProperties(
+      @PathParam("setname") String setname,
+
       @PathParam("config") String config,
       @Context UriInfo uriInfo,
       @Context HttpServletResponse res) throws IOException {
 
-    return getStringRepresentation(config, uriInfo, res,
+    return getStringRepresentation(setname, config, uriInfo, res,
         ConfigFormat.PROPERTIES);
   }
 
-  public String getStringRepresentation(String config,
+  public String getStringRepresentation(String setname,
+      String config,
       UriInfo uriInfo,
       HttpServletResponse res, ConfigFormat format) throws IOException {
     // delegate (including init)
     PublishedConfiguration publishedConfig =
-        getConfigurationInstance(config, uriInfo, res);
+        getConfigurationInstance(setname, config, uriInfo, res);
     PublishedConfigurationOutputter outputter =
         publishedConfig.createOutputter(format);
     return outputter.asString();
   }
 
   @GET
-  @Path("/{config}/{propertyName}")
+  @Path("/" + CONFIG +"/{propertyName}")
   @Produces({MediaType.APPLICATION_JSON})
   public Map<String,String> getConfigurationProperty(
-      @PathParam("propertyName") String propertyName,
+      @PathParam("setname") String setname,
       @PathParam("config") String config,
+      @PathParam("propertyName") String propertyName,
       @Context UriInfo uriInfo,
       @Context HttpServletResponse res) {
     PublishedConfiguration publishedConfig =
-        getConfigurationInstance(config, uriInfo, res);
+        getConfigurationInstance(setname, config, uriInfo, res);
     String propVal = publishedConfig.entries.get(propertyName);
     if (propVal == null) {
-      log.info("Configuration property {} not found in configuration {}",
-               propertyName, config);
+      log.debug("Configuration property {} not found in configuration {}",
+          propertyName, config);
       throw new NotFoundException("Property not found: " + propertyName);
     }
     Map<String,String> rtnVal = new HashMap<>();

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java Tue May 20 18:27:20 2014
@@ -30,7 +30,7 @@ import org.slf4j.LoggerFactory;
 import java.io.Closeable;
 
 public class CuratorService extends AbstractService {
-  protected static final Logger log =
+  private static final Logger log =
     LoggerFactory.getLogger(CuratorService.class);
   protected final String basePath;
 

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java Tue May 20 18:27:20 2014
@@ -28,7 +28,6 @@ import org.apache.curator.x.discovery.Se
 import org.apache.curator.x.discovery.ServiceType;
 import org.apache.curator.x.discovery.UriSpec;
 import org.apache.slider.core.exceptions.BadClusterStateException;
-import org.apache.slider.core.exceptions.UnknownApplicationInstanceException;
 import org.apache.slider.core.persist.JsonSerDeser;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -49,7 +48,7 @@ import java.util.Map;
  * @param <Payload> the payload of the operation
  */
 public class RegistryBinderService<Payload> extends CuratorService {
-  protected static final Logger log =
+  private static final Logger log =
     LoggerFactory.getLogger(RegistryBinderService.class);
 
   private final ServiceDiscovery<Payload> discovery;
@@ -57,12 +56,13 @@ public class RegistryBinderService<Paylo
   private final Map<String, ServiceInstance<Payload>> entries =
     new HashMap<>();
 
-  JsonSerDeser<CuratorServiceInstance<Payload>> deser =
+  private JsonSerDeser<CuratorServiceInstance<Payload>> deser =
     new JsonSerDeser<>(CuratorServiceInstance.class);
 
   /**
    * Create an instance
-   * @param curator. Again, does not need to be started
+   * @param curator  Does not need to be started
+   * @param basePath base directory
    * @param discovery discovery instance -not yet started
    */
   public RegistryBinderService(CuratorFramework curator,
@@ -106,7 +106,6 @@ public class RegistryBinderService<Paylo
                                            Payload payload) throws Exception {
     Preconditions.checkNotNull(id, "null `id` arg");
     Preconditions.checkNotNull(name, "null `name` arg");
-    Preconditions.checkNotNull(url, "null `url` arg");
     Preconditions.checkState(isInState(STATE.STARTED), "Not started: " + this);
 
     if (lookup(id) != null) {
@@ -114,19 +113,24 @@ public class RegistryBinderService<Paylo
         "existing entry for service id %s name %s %s",
         id, name, url);
     }
-    int port = url.getPort();
-    if (port == 0) {
-      throw new IOException("Port undefined in " + url);
-    }
-    UriSpec uriSpec = new UriSpec(url.toString());
-    ServiceInstance<Payload> instance = builder()
-      .name(name)
-      .id(id)
-      .payload(payload)
-      .port(port)
-      .serviceType(ServiceType.DYNAMIC)
-      .uriSpec(uriSpec)
-      .build();
+
+    ServiceInstanceBuilder<Payload> instanceBuilder = builder()
+        .name(name)
+        .id(id)
+        .payload(payload)
+        .serviceType(ServiceType.DYNAMIC);
+    if (url != null) {
+      UriSpec uriSpec = new UriSpec(url.toString());
+
+      int port = url.getPort();
+      if (port == 0) {
+        throw new IOException("Port undefined in " + url);
+      }
+      instanceBuilder
+          .uriSpec(uriSpec)
+          .port(port);
+    }
+    ServiceInstance<Payload> instance = instanceBuilder.build();
     log.info("registering {}", instance.toString());
     discovery.registerService(instance);
     log.info("registration completed {}", instance.toString());
@@ -184,7 +188,7 @@ public class RegistryBinderService<Paylo
 
   /**
    * List all service types registered
-   * @return
+   * @return a list of service types
    * @throws Exception
    */
   public List<String> serviceTypes() throws Exception {
@@ -296,7 +300,12 @@ public class RegistryBinderService<Paylo
     return instances;
   }
 
-  public Collection<String> queryForNames() throws IOException {
+  /**
+   * Enum all service types in the registry
+   * @return a possibly empty collection of service types
+   * @throws IOException networking
+   */
+  public Collection<String> getServiceTypes() throws IOException {
     try {
       return getDiscovery().queryForNames();
     } catch (IOException e) {

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryViewForProviders.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryViewForProviders.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryViewForProviders.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryViewForProviders.java Tue May 20 18:27:20 2014
@@ -21,6 +21,7 @@ package org.apache.slider.server.service
 import org.apache.slider.core.registry.info.ServiceInstanceData;
 
 import java.io.IOException;
+import java.net.URL;
 import java.util.List;
 
 /**
@@ -35,4 +36,18 @@ public interface RegistryViewForProvider
    * @return the registration of slider
    */
   ServiceInstanceData getSelfRegistration();
+
+  /**
+   * Register the service, raising IOExceptions when anything fails
+   * @param serviceType service type
+   * @param instanceName ID -must be unique
+   * @param url URL to register
+   * @param instanceData instance data
+   * @throws IOException on registration problems
+   */
+  void registerServiceInstance(
+      String serviceType,
+      String instanceName,
+      URL url,
+      ServiceInstanceData instanceData) throws IOException;
 }

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/SliderRegistryService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/SliderRegistryService.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/SliderRegistryService.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/SliderRegistryService.java Tue May 20 18:27:20 2014
@@ -18,6 +18,7 @@
 
 package org.apache.slider.server.services.registry;
 
+import com.google.common.base.Preconditions;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.x.discovery.ServiceDiscovery;
 import org.apache.slider.core.registry.info.ServiceInstanceData;
@@ -37,7 +38,7 @@ import java.util.List;
 public class SliderRegistryService
     extends RegistryBinderService<ServiceInstanceData>
     implements RegistryViewForProviders {
-  
+
   private ServiceInstanceData selfRegistration;
 
   public SliderRegistryService(CuratorFramework curator,
@@ -71,20 +72,29 @@ public class SliderRegistryService
   /**
    * register an instance -only valid once the service is started.
    * This sets the selfRegistration field
-   * @param id ID -must be unique
-   * @param name name
-   * @param url URL
-   * @param payload payload (may be null)
-   * @return the instance
+   * @param serviceType service type
+   * @param instanceName ID -must be unique
+   * @param url URL to register
+   * @param instanceData instance data
    * @throws IOException on registration problems
    */
-  public void registerSelf(String name,
-      String id,
+  public void registerSelf(String serviceType,
+      String instanceName,
+      URL url,
+      ServiceInstanceData instanceData) throws IOException {
+    registerServiceInstance(serviceType, instanceName, url, instanceData);
+    setSelfRegistration(instanceData);
+  }
+
+  @Override
+  public void registerServiceInstance(
+      String serviceType,
+      String instanceName,
       URL url,
       ServiceInstanceData instanceData) throws IOException {
+    Preconditions.checkNotNull(instanceData);
     try {
-      register(name, id, url, instanceData);
-      setSelfRegistration(instanceData);
+      register(serviceType, instanceName, url, instanceData);
     } catch (IOException e) {
       throw e;
     } catch (Exception e) {

Added: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/utility/PatternValidator.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/utility/PatternValidator.java?rev=1596353&view=auto
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/utility/PatternValidator.java (added)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/utility/PatternValidator.java Tue May 20 18:27:20 2014
@@ -0,0 +1,49 @@
+/*
+ * 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.services.utility;
+
+import org.apache.slider.server.appmaster.web.rest.RestPaths;
+
+import java.util.regex.Pattern;
+
+public class PatternValidator {
+
+  public static final String E_INVALID_NAME =
+      "Invalid name %s does not match the pattern pattern %s ";
+  private final Pattern valid;
+  private final String pattern;
+
+  public PatternValidator(String pattern) {
+    this.pattern = pattern;
+    valid = Pattern.compile(pattern);
+  }
+
+  /**
+   * Validate the name -restricting it to the set defined in 
+   * {@link RestPaths#PUBLISHED_CONFIGURATION_REGEXP}
+   * @param name name to validate
+   * @throws IllegalArgumentException if not a valid name
+   */
+  public void validate(String name) {
+    if (!valid.matcher(name).matches()) {
+      throw new IllegalArgumentException(
+          String.format(E_INVALID_NAME, name, pattern));
+    }
+  }
+}

Propchange: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/utility/PatternValidator.java
------------------------------------------------------------------------------
    svn:eol-style = native

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=1596353&r1=1596352&r2=1596353&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 Tue May 20 18:27:20 2014
@@ -97,7 +97,7 @@ class TestStandaloneAgentAM  extends Age
 
     describe "service registry names"
     SliderRegistryService registry = client.registry
-    def names = registry.queryForNames();
+    def names = registry.getServiceTypes();
     dumpRegistryNames(names)
     describe "service registry instance IDs"
 

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=1596353&r1=1596352&r2=1596353&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 Tue May 20 18:27:20 2014
@@ -33,10 +33,12 @@ import org.apache.slider.core.main.Servi
 import org.apache.slider.core.persist.JsonSerDeser
 import org.apache.slider.core.registry.docstore.PublishedConfigSet
 import org.apache.slider.core.registry.docstore.PublishedConfiguration
+import org.apache.slider.core.registry.docstore.UriMap
 import org.apache.slider.core.registry.info.CustomRegistryConstants
 import org.apache.slider.core.registry.info.ServiceInstanceData
 import org.apache.slider.core.registry.retrieve.RegistryRetriever
 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.registry.SliderRegistryService
 import org.junit.Test
@@ -112,8 +114,8 @@ class TestStandaloneRegistryAM extends A
 
     describe "service registry names"
     SliderRegistryService registryService = client.registry
-    def names = registryService.queryForNames();
-    dumpRegistryNames(names)
+    def serviceTypes = registryService.serviceTypes;
+    dumpRegistryNames(serviceTypes)
 
     List<String> instanceIds = client.listRegistryInstanceIDs()
 
@@ -146,8 +148,14 @@ class TestStandaloneRegistryAM extends A
     def publisher = publisherURL.toString()
     describe("Publisher")
 
-    def publishedJSON = GET(publisherURL)
-//    log.info(publishedJSON)
+    JsonSerDeser<UriMap> uriMapDeser = new JsonSerDeser<>(UriMap)
+    def setlisting = GET(publisherURL)
+
+    log.info(setlisting)
+
+    UriMap uris = uriMapDeser.fromJson(setlisting)
+    assert uris.uris[RestPaths.SLIDER_CONFIGSET]
+    def publishedJSON = GET(publisherURL, RestPaths.SLIDER_CONFIGSET)
     JsonSerDeser< PublishedConfigSet> serDeser= new JsonSerDeser<>(
         PublishedConfigSet)
     def configSet = serDeser.fromJson(publishedJSON)
@@ -158,7 +166,9 @@ class TestStandaloneRegistryAM extends A
     assert publishedYarnSite.empty
     
     //get the full URL
-    def yarnSitePublisher = appendToURL(publisher, ARTIFACT_NAME)
+    def yarnSitePublisher = appendToURL(publisher,
+        RestPaths.SLIDER_CONFIGSET,
+        ARTIFACT_NAME)
 
     String confJSON = GET(yarnSitePublisher)
 //    log.info(confJSON)
@@ -202,10 +212,7 @@ class TestStandaloneRegistryAM extends A
     
     assert retriever.hasConfigurations(true)
     PublishedConfigSet externalConfSet = retriever.getConfigurations(true)
-    externalConfSet.keys().each { String key ->
-      def config = externalConfSet.get(key)
-      log.info "$key -- ${config.description}"
-    }
+    dumpConfigurationSet(externalConfSet)
     assert externalConfSet[ARTIFACT_NAME]
 
 
@@ -221,8 +228,6 @@ class TestStandaloneRegistryAM extends A
     assert rmHostnameViaClientSideXML == rmHostnameFromDownloadedProperties
     def rmAddrViaClientSideXML = siteXML.get(YarnConfiguration.RM_ADDRESS)
 
-  //TODO SLIDER-52 PublishedConfiguration XML conf values are not resolved until client-side
-    
     log.info("RM from downloaded props = $rmAddrFromDownloadedProperties")
     assert rmAddrViaClientSideXML == rmAddrFromDownloadedProperties
     
@@ -355,6 +360,4 @@ class TestStandaloneRegistryAM extends A
     assert instances.size() == 0
 
   }
-
-
 }

Modified: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy Tue May 20 18:27:20 2014
@@ -31,6 +31,7 @@ import org.apache.slider.server.appmaste
 import org.apache.slider.server.appmaster.model.mock.MockSliderClusterProtocol
 import org.apache.slider.server.appmaster.state.AbstractRecordFactory
 import org.apache.slider.server.appmaster.state.AppState
+import org.apache.slider.server.appmaster.state.ProviderAppState
 import org.apache.slider.server.appmaster.web.WebAppApi
 import org.apache.slider.server.appmaster.web.WebAppApiImpl
 import org.junit.Before
@@ -46,9 +47,12 @@ public class TestClusterSpecificationBlo
   public void setup() {
     SliderClusterProtocol clusterProto = new MockSliderClusterProtocol();
     AppState appState = new MyAppState(new MockRecordFactory());
+    ProviderAppState providerAppState = new ProviderAppState(
+        "undefined",
+        appState)
     ProviderService providerService = new MockProviderService();
 
-    WebAppApiImpl inst = new WebAppApiImpl(clusterProto, appState, providerService);
+    WebAppApiImpl inst = new WebAppApiImpl(clusterProto, providerAppState, providerService);
 
     Injector injector = Guice.createInjector(new AbstractModule() {
           @Override

Modified: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy Tue May 20 18:27:20 2014
@@ -33,6 +33,7 @@ import org.apache.slider.api.SliderClust
 import org.apache.slider.providers.ProviderService
 import org.apache.slider.server.appmaster.model.mock.*
 import org.apache.slider.server.appmaster.state.AppState
+import org.apache.slider.server.appmaster.state.ProviderAppState
 import org.apache.slider.server.appmaster.state.RoleInstance
 import org.apache.slider.server.appmaster.web.WebAppApi
 import org.apache.slider.server.appmaster.web.WebAppApiImpl
@@ -56,8 +57,11 @@ public class TestContainerStatsBlock {
     SliderClusterProtocol clusterProto = new MockSliderClusterProtocol();
     AppState appState = new MockAppState(new MockRecordFactory());
     ProviderService providerService = new MockProviderService();
+    ProviderAppState providerAppState = new ProviderAppState(
+        "undefined",
+        appState)
 
-    WebAppApiImpl inst = new WebAppApiImpl(clusterProto, appState, providerService);
+    WebAppApiImpl inst = new WebAppApiImpl(clusterProto, providerAppState, providerService);
 
     Injector injector = Guice.createInjector(new AbstractModule() {
           @Override

Modified: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy Tue May 20 18:27:20 2014
@@ -28,6 +28,7 @@ import org.apache.slider.api.SliderClust
 import org.apache.slider.providers.ProviderService
 import org.apache.slider.server.appmaster.model.mock.*
 import org.apache.slider.server.appmaster.state.AppState
+import org.apache.slider.server.appmaster.state.ProviderAppState
 import org.apache.slider.server.appmaster.web.WebAppApi
 import org.apache.slider.server.appmaster.web.WebAppApiImpl
 import org.junit.Before
@@ -46,8 +47,11 @@ public class TestIndexBlock {
     SliderClusterProtocol clusterProto = new MockSliderClusterProtocol();
     AppState appState = new MockAppState(new MockRecordFactory());
     ProviderService providerService = new MockProviderService();
+    ProviderAppState providerAppState = new ProviderAppState(
+        "undefined",
+        appState)
 
-    WebAppApiImpl inst = new WebAppApiImpl(clusterProto, appState, providerService);
+    WebAppApiImpl inst = new WebAppApiImpl(clusterProto, providerAppState, providerService);
 
     Injector injector = Guice.createInjector(new AbstractModule() {
           @Override

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=1596353&r1=1596352&r2=1596353&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 Tue May 20 18:27:20 2014
@@ -44,6 +44,7 @@ import org.apache.slider.core.exceptions
 import org.apache.slider.core.main.ServiceLaunchException
 import org.apache.slider.core.main.ServiceLauncher
 import org.apache.slider.core.persist.JsonSerDeser
+import org.apache.slider.core.registry.docstore.PublishedConfigSet
 import org.apache.slider.core.registry.info.ServiceInstanceData
 import org.apache.slider.server.services.curator.CuratorServiceInstance
 import org.junit.Assert
@@ -377,6 +378,10 @@ class SliderTestUtils extends Assert {
     return SliderUtils.appendToURL(base, path)
   }
 
+  public static String appendToURL(String base, String... paths) {
+    return SliderUtils.appendToURL(base, paths)
+  }
+
   /**
    * Fetch a web page 
    * @param url URL
@@ -718,4 +723,11 @@ class SliderTestUtils extends Assert {
     }
     return time;
   }
+
+  def dumpConfigurationSet(PublishedConfigSet confSet) {
+    confSet.keys().each { String key ->
+      def config = confSet.get(key)
+      log.info "$key -- ${config.description}"
+    }
+  }
 }

Modified: incubator/slider/trunk/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java (original)
+++ incubator/slider/trunk/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java Tue May 20 18:27:20 2014
@@ -18,9 +18,6 @@
 
 package org.apache.slider.providers.agent;
 
-import org.apache.slider.server.appmaster.web.rest.agent.CommandReport;
-import org.junit.Assert;
-import org.junit.Test;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FilterFileSystem;
 import org.apache.hadoop.fs.Path;
@@ -56,14 +53,16 @@ import org.apache.slider.providers.agent
 import org.apache.slider.server.appmaster.model.mock.MockContainerId;
 import org.apache.slider.server.appmaster.model.mock.MockFileSystem;
 import org.apache.slider.server.appmaster.model.mock.MockNodeId;
-import org.apache.slider.server.appmaster.state.AppState;
+import org.apache.slider.server.appmaster.state.ProviderAppState;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.apache.slider.server.appmaster.web.rest.agent.CommandReport;
 import org.apache.slider.server.appmaster.web.rest.agent.ComponentStatus;
 import org.apache.slider.server.appmaster.web.rest.agent.HeartBeat;
 import org.apache.slider.server.appmaster.web.rest.agent.HeartBeatResponse;
 import org.apache.slider.server.appmaster.web.rest.agent.Register;
 import org.apache.slider.server.appmaster.web.rest.agent.RegistrationResponse;
 import org.apache.slider.server.appmaster.web.rest.agent.RegistrationStatus;
+import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Mockito;
@@ -204,7 +203,7 @@ public class TestAgentProviderService {
     StateAccessForProviders access = createNiceMock(StateAccessForProviders.class);
 
     AgentProviderService mockAps = Mockito.spy(aps);
-    doReturn(access).when(mockAps).getStateAccessor();
+    doReturn(access).when(mockAps).getAmState();
     doReturn("scripts/hbase_master.py").when(mockAps).getScriptPathFromMetainfo(anyString());
     Metainfo metainfo = new Metainfo();
     metainfo.addService(new Service());
@@ -244,10 +243,8 @@ public class TestAgentProviderService {
                                           resourceComponent,
                                           appComponent,
                                           containerTmpDirPath);
-    } catch (SliderException he) {
-      log.warn(he.getMessage());
-    } catch (IOException ioe) {
-      log.warn(ioe.getMessage());
+    } catch (SliderException | IOException he) {
+      log.warn("{}", he, he);
     }
 
     Register reg = new Register();
@@ -267,7 +264,7 @@ public class TestAgentProviderService {
   @Test
   public void testRoleHostMapping() throws Exception {
     AgentProviderService aps = new AgentProviderService();
-    StateAccessForProviders appState = new AppState(null) {
+    StateAccessForProviders appState = new ProviderAppState("undefined", null) {
       @Override
       public ClusterDescription getClusterStatus() {
         ClusterDescription cd = new ClusterDescription();
@@ -304,8 +301,8 @@ public class TestAgentProviderService {
       }
     };
 
-    aps.setStateAccessor(appState);
-    Map<String, String> tokens = new HashMap<String, String>();
+    aps.setAmState(appState);
+    Map<String, String> tokens = new HashMap<>();
     aps.addRoleRelatedTokens(tokens);
     Assert.assertEquals(2, tokens.size());
     Assert.assertEquals("FIRST_HOST", tokens.get("${FIRST_ROLE_HOST}"));
@@ -486,7 +483,7 @@ public class TestAgentProviderService {
     StateAccessForProviders access = createNiceMock(StateAccessForProviders.class);
 
     AgentProviderService mockAps = Mockito.spy(aps);
-    doReturn(access).when(mockAps).getStateAccessor();
+    doReturn(access).when(mockAps).getAmState();
     doReturn(metainfo).when(mockAps).getApplicationMetainfo(any(SliderFileSystem.class), anyString());
 
     try {
@@ -662,10 +659,8 @@ public class TestAgentProviderService {
                                                                 anyString(),
                                                                 any(HeartBeatResponse.class),
                                                                 anyString());
-    } catch (SliderException he) {
+    } catch (SliderException | IOException he) {
       log.warn(he.getMessage());
-    } catch (IOException ioe) {
-      log.warn(ioe.getMessage());
     }
   }
 
@@ -677,7 +672,7 @@ public class TestAgentProviderService {
 
     StateAccessForProviders access = createNiceMock(StateAccessForProviders.class);
     AgentProviderService mockAps = Mockito.spy(aps);
-    doReturn(access).when(mockAps).getStateAccessor();
+    doReturn(access).when(mockAps).getAmState();
 
     AggregateConf aggConf = new AggregateConf();
     ConfTreeOperations treeOps = aggConf.getAppConfOperations();

Modified: incubator/slider/trunk/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java (original)
+++ incubator/slider/trunk/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java Tue May 20 18:27:20 2014
@@ -43,6 +43,7 @@ import org.apache.slider.server.appmaste
 import org.apache.slider.server.appmaster.model.mock.MockRecordFactory;
 import org.apache.slider.server.appmaster.model.mock.MockSliderClusterProtocol;
 import org.apache.slider.server.appmaster.state.AppState;
+import org.apache.slider.server.appmaster.state.ProviderAppState;
 import org.apache.slider.server.appmaster.web.WebAppApi;
 import org.apache.slider.server.appmaster.web.WebAppApiImpl;
 import org.apache.slider.server.appmaster.web.rest.AMWebServices;
@@ -138,7 +139,10 @@ public class TestAMAgentWebServices exte
         } catch (Exception e) {
           log.error("Failed to set up app {}", e);
         }
-        slider = new WebAppApiImpl(new MockSliderClusterProtocol(), appState,
+        ProviderAppState providerAppState = new ProviderAppState("undefined",
+            appState);
+
+        slider = new WebAppApiImpl(new MockSliderClusterProtocol(), providerAppState,
                                    new MockProviderService());
 
         bind(SliderJacksonJaxbJsonProvider.class);

Modified: incubator/slider/trunk/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java?rev=1596353&r1=1596352&r2=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java (original)
+++ incubator/slider/trunk/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java Tue May 20 18:27:20 2014
@@ -43,6 +43,7 @@ import org.apache.slider.server.appmaste
 import org.apache.slider.server.appmaster.model.mock.MockRecordFactory;
 import org.apache.slider.server.appmaster.model.mock.MockSliderClusterProtocol;
 import org.apache.slider.server.appmaster.state.AppState;
+import org.apache.slider.server.appmaster.state.ProviderAppState;
 import org.apache.slider.server.appmaster.web.WebAppApi;
 import org.apache.slider.server.appmaster.web.WebAppApiImpl;
 import org.apache.slider.server.appmaster.web.rest.AMWebServices;
@@ -51,6 +52,8 @@ import org.codehaus.jettison.json.JSONEx
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.ws.rs.Path;
 import javax.ws.rs.core.MediaType;
@@ -64,7 +67,8 @@ import static org.junit.Assert.assertNot
 import static org.junit.Assert.fail;
 
 public class TestAMManagementWebServices extends JerseyTest {
-
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestAMManagementWebServices.class);
   public static final int RM_MAX_RAM = 4096;
   public static final int RM_MAX_CORES = 64;
   public static final String EXAMPLES =
@@ -169,16 +173,13 @@ public class TestAMManagementWebServices
               fs,
               historyPath,
               null, null);
-        } catch (IOException e) {
-          e.printStackTrace();  //To change body of catch statement use File | Settings | File Templates.
-        } catch (URISyntaxException e) {
-          e.printStackTrace();  //To change body of catch statement use File | Settings | File Templates.
-        } catch (BadClusterStateException e) {
-          e.printStackTrace();  //To change body of catch statement use File | Settings | File Templates.
-        } catch (BadConfigException e) {
-          e.printStackTrace();  //To change body of catch statement use File | Settings | File Templates.
+        } catch (IOException | BadClusterStateException | URISyntaxException | BadConfigException e) {
+          log.error("{}", e, e);
         }
-        slider = new WebAppApiImpl(new MockSliderClusterProtocol(), appState,
+        ProviderAppState providerAppState = new ProviderAppState("undefined",
+            appState);
+
+        slider = new WebAppApiImpl(new MockSliderClusterProtocol(), providerAppState,
                                    new MockProviderService());
 
         bind(SliderJacksonJaxbJsonProvider.class);

Copied: incubator/slider/trunk/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentClusterLifecycle.groovy (from r1596135, incubator/slider/trunk/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestClusterLifecycle.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentClusterLifecycle.groovy?p2=incubator/slider/trunk/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentClusterLifecycle.groovy&p1=incubator/slider/trunk/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestClusterLifecycle.groovy&r1=1596135&r2=1596353&rev=1596353&view=diff
==============================================================================
--- incubator/slider/trunk/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestClusterLifecycle.groovy (original)
+++ incubator/slider/trunk/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentClusterLifecycle.groovy Tue May 20 18:27:20 2014
@@ -34,11 +34,11 @@ import org.junit.Test
 
 @CompileStatic
 @Slf4j
-public class TestClusterLifecycle extends AgentCommandTestBase
+public class TestAgentClusterLifecycle extends AgentCommandTestBase
     implements FuntestProperties, Arguments, SliderExitCodes {
 
 
-  static String CLUSTER = "test_cluster_lifecycle"
+  static String CLUSTER = "test_agent_cluster_lifecycle"
 
 
   @Before
@@ -52,7 +52,7 @@ public class TestClusterLifecycle extend
   }
 
   @Test
-  public void testClusterLifecycle() throws Throwable {
+  public void testAgentClusterLifecycle() throws Throwable {
 
     describe "Walk a 0-role cluster through its lifecycle"