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/06 20:33:35 UTC

incubator-slider git commit: SLIDER-741 expose raw/resolved views of app configurations

Repository: incubator-slider
Updated Branches:
  refs/heads/develop 566b3f66f -> 0d2a35bb6


SLIDER-741 expose raw/resolved views of app configurations


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

Branch: refs/heads/develop
Commit: 0d2a35bb637155bc9f88e662966a1a2246ed0da0
Parents: 566b3f6
Author: Steve Loughran <st...@apache.org>
Authored: Tue Jan 6 19:32:57 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Jan 6 19:32:57 2015 +0000

----------------------------------------------------------------------
 .../core/persist/AggregateConfSerDeser.java     |  55 +++++++
 .../server/appmaster/SliderAppMaster.java       |   4 +-
 .../slider/server/appmaster/state/AppState.java |  87 ++++++++---
 .../appmaster/state/ProviderAppState.java       |   5 +
 .../state/StateAccessForProviders.java          |  10 ++
 .../server/appmaster/web/rest/RestPaths.java    |   5 +-
 .../rest/application/ApplicationResource.java   | 143 +++++++++++++++----
 .../resources/AggregateModelRefresher.java      |  47 ++++++
 .../application/resources/AppconfRefresher.java |  55 +++++++
 .../application/resources/CachedContent.java    |   6 +-
 .../application/resources/ContentCache.java     |   3 +-
 .../resources/LiveComponentsRefresher.java      |   3 +-
 .../resources/LiveContainersRefresher.java      |   3 +-
 .../resources/LiveResourcesRefresher.java       |   2 +-
 .../resources/ResourceRefresher.java            |   2 +-
 .../resources/ResourceSnapshotRefresher.java    |   2 +-
 .../standalone/TestStandaloneAgentWeb.groovy    |  64 ++++++++-
 .../TestMockAppStateAppRestIntegration.groovy   |   2 +-
 .../apache/slider/test/SliderTestUtils.groovy   |  17 +++
 19 files changed, 445 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/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 c8764c9..0dd6c59 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
@@ -596,8 +596,8 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     instanceDefinition.getAppConfOperations().set(
         KEY_SECURITY_ENABLED, securityEnabled);
 
-    // triggers resolution and snapshotting in agent
-    appState.updateInstanceDefinition(instanceDefinition);
+    // triggers resolution and snapshotting for agent
+    appState.setInitialInstanceDefinition(instanceDefinition);
 
     File confDir = getLocalConfDir();
     if (!confDir.exists() || !confDir.isDirectory()) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
index 9ae20a5..f381cde 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
@@ -59,6 +59,8 @@ import org.apache.slider.core.exceptions.ErrorStrings;
 import org.apache.slider.core.exceptions.NoSuchNodeException;
 import org.apache.slider.core.exceptions.SliderInternalStateException;
 import org.apache.slider.core.exceptions.TriggerClusterTeardownException;
+import org.apache.slider.core.persist.AggregateConfSerDeser;
+import org.apache.slider.core.persist.ConfTreeSerDeser;
 import org.apache.slider.providers.PlacementPolicy;
 import org.apache.slider.providers.ProviderRole;
 import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
@@ -109,16 +111,30 @@ public class AppState {
 
   /**
    * The definition of the instance. Flexing updates the resources section
-   This is used as a synchronization point on activities that update
-   the CD, and also to update some of the structures that
-   feed in to the CD
+   * This is used as a synchronization point on activities that update
+   * the CD, and also to update some of the structures that
+   * feed in to the CD
    */
   private AggregateConf instanceDefinition;
-  
+
+  /**
+   * Time the instance definition snapshots were created
+   */
   private long snapshotTime;
+
+  /**
+   * Snapshot of the instance definition. This is fully
+   * resolved.
+   */
   private AggregateConf instanceDefinitionSnapshot;
 
   /**
+   * Snapshot of the raw instance definition; unresolved and
+   * without any patch of an AM into it.
+   */
+  private AggregateConf unresolvedInstanceDefinition;
+
+  /**
    * snapshot of resources as of last update time
    */
   private ConfTreeOperations resourcesSnapshot;
@@ -379,13 +395,18 @@ public class AppState {
    * 
    * Important: this is for early binding and must not be used after the build
    * operation is complete. 
-   * @param definition
+   * @param definition initial definition
    * @throws BadConfigException
    */
-  public synchronized void updateInstanceDefinition(AggregateConf definition) throws
-                                                                              BadConfigException,
-                                                                              IOException {
-    this.instanceDefinition = definition;
+  public synchronized void setInitialInstanceDefinition(AggregateConf definition)
+      throws BadConfigException, IOException {
+    log.debug("Setting initial instance definition");
+    // snapshot the definition
+    AggregateConfSerDeser serDeser = new AggregateConfSerDeser();
+
+    unresolvedInstanceDefinition = serDeser.fromInstance(definition);
+    
+    this.instanceDefinition = serDeser.fromInstance(definition);
     onInstanceDefinitionUpdated();
   }
 
@@ -453,6 +474,10 @@ public class AppState {
     return instanceDefinitionSnapshot;
   }
 
+  public AggregateConf getUnresolvedInstanceDefinition() {
+    return unresolvedInstanceDefinition;
+  }
+
   /**
    * Build up the application state
    * @param instanceDefinition definition of the applicatin instance
@@ -462,8 +487,8 @@ public class AppState {
    * @param fs filesystem
    * @param historyDir directory containing history files
    * @param liveContainers list of live containers supplied on an AM restart
-   * @param applicationInfo
-   * @param releaseSelector
+   * @param applicationInfo app info to retain for web views
+   * @param releaseSelector selector of containers to release
    */
   public synchronized void buildInstance(AggregateConf instanceDefinition,
       Configuration appmasterConfig,
@@ -478,6 +503,7 @@ public class AppState {
     Preconditions.checkArgument(instanceDefinition != null);
     Preconditions.checkArgument(releaseSelector != null);
 
+    log.debug("Building application state");
     this.publishedProviderConf = publishedProviderConf;
     this.applicationInfo = applicationInfo != null ? applicationInfo
                                                    : new HashMap<String, String>();
@@ -496,7 +522,7 @@ public class AppState {
 
     // set the cluster specification (once its dependency the client properties
     // is out the way
-    updateInstanceDefinition(instanceDefinition);
+    setInitialInstanceDefinition(instanceDefinition);
 
     //build the initial role list
     for (ProviderRole providerRole : providerRoles) {
@@ -544,8 +570,8 @@ public class AppState {
     rebuildModelFromRestart(liveContainers);
 
     // any am config options to pick up
-
     logServerURL = appmasterConfig.get(YarnConfiguration.YARN_LOG_SERVER_URL, "");
+    
     //mark as live
     applicationLive = true;
   }
@@ -607,13 +633,26 @@ public class AppState {
 
   /**
    * Actions to perform when an instance definition is updated
-   * Currently: resolve the configuration
-   *  updated the cluster spec derivative
+   * Currently: 
+   * <ol>
+   *   <li>
+   *     resolve the configuration
+   *   </li>
+   *   <li>
+   *     update the cluster spec derivative
+   *   </li>
+   * </ol>
+   *  
    * @throws BadConfigException
    */
   private synchronized void onInstanceDefinitionUpdated() throws
                                                           BadConfigException,
                                                           IOException {
+    log.debug("Instance definition updated");
+    //note the time 
+    snapshotTime = now();
+    
+    // resolve references if not already done
     instanceDefinition.resolve();
 
     // force in the AM desired state values
@@ -624,8 +663,7 @@ public class AppState {
           SliderKeys.COMPONENT_AM, ResourceKeys.COMPONENT_INSTANCES, "1");
     }
 
-    //note the time 
-    snapshotTime = now();
+
     //snapshot all three sectons
     resourcesSnapshot =
       ConfTreeOperations.fromInstance(instanceDefinition.getResources());
@@ -660,11 +698,16 @@ public class AppState {
   public synchronized List<ProviderRole> updateResourceDefinitions(ConfTree resources)
       throws BadConfigException, IOException {
     log.debug("Updating resources to {}", resources);
-    
-    instanceDefinition.setResources(resources);
+    // snapshot the (possibly unresolved) values
+    ConfTreeSerDeser serDeser = new ConfTreeSerDeser();
+    unresolvedInstanceDefinition.setResources(
+        serDeser.fromInstance(resources));
+    // assign another copy under the instance definition for resolving
+    // and then driving application size
+    instanceDefinition.setResources(serDeser.fromInstance(resources));
     onInstanceDefinitionUpdated();
  
-    //propagate the role table
+    // propagate the role table
     Map<String, Map<String, String>> updated = resources.components;
     getClusterStatus().roles = SliderUtils.deepClone(updated);
     getClusterStatus().updateTime = now();
@@ -952,14 +995,12 @@ public class AppState {
     return findNodeInCollection(containerId, nodes);
   }
 
-  
-  
   /**
    * Iterate through a collection of role instances to find one with a
    * specific (string) container ID
    * @param containerId container ID as a string
    * @param nodes collection
-   * @return 
+   * @return the found node 
    * @throws NoSuchNodeException if there was no match
    */
   private RoleInstance findNodeInCollection(String containerId,

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java
index d5a041b..43ff52f 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java
@@ -154,6 +154,11 @@ public class ProviderAppState implements StateAccessForProviders {
   public AggregateConf getInstanceDefinitionSnapshot() {
     return appState.getInstanceDefinitionSnapshot();
   }
+  
+  @Override
+  public AggregateConf getUnresolvedInstanceDefinition() {
+    return appState.getUnresolvedInstanceDefinition();
+  }
 
   @Override
   public RoleStatus lookupRoleStatus(int key) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
index 75076ed..2cbe6b1 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
@@ -117,9 +117,19 @@ public interface StateAccessForProviders {
 
   long getSnapshotTime();
 
+  /**
+   * Get a snapshot of the entire aggregate configuration
+   * @return the aggregate configuration
+   */
   AggregateConf getInstanceDefinitionSnapshot();
 
   /**
+   * Get the desired/unresolved value
+   * @return unresolved
+   */
+  AggregateConf getUnresolvedInstanceDefinition();
+
+  /**
    * Look up a role from its key -or fail 
    *
    * @param key key to resolve

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
index 011ec3a..f637190 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
@@ -134,10 +134,11 @@ public class RestPaths {
   public static final String LIVE_RESOURCES = "/live/resources";
   public static final String LIVE_CONTAINERS = "/live/containers";
   public static final String LIVE_COMPONENTS = "/live/components";
-  public static final String MODEL_DESIRED = "/model/desired";
+  public static final String MODEL = "/model";
+  public static final String MODEL_DESIRED = MODEL +"/desired";
   public static final String MODEL_DESIRED_APPCONF = MODEL_DESIRED +"/appconf";
   public static final String MODEL_DESIRED_RESOURCES = MODEL_DESIRED +"/resources";
-  public static final String MODEL_RESOLVED = "/model/desired";
+  public static final String MODEL_RESOLVED = "/model/resolved";
   public static final String MODEL_RESOLVED_APPCONF = MODEL_RESOLVED +"/appconf";
   public static final String MODEL_RESOLVED_RESOURCES = MODEL_RESOLVED +"/resources";
   public static final String MODEL_INTERNAL = "/model/internal";

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java
index 9419765..48540d4 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java
@@ -19,11 +19,11 @@
 package org.apache.slider.server.appmaster.web.rest.application;
 
 import com.google.common.collect.Lists;
-import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 import org.apache.slider.api.types.SerializedComponentInformation;
 import org.apache.slider.api.types.SerializedContainerInformation;
+import org.apache.slider.core.conf.AggregateConf;
 import org.apache.slider.core.conf.ConfTree;
 import org.apache.slider.core.exceptions.NoSuchNodeException;
 import org.apache.slider.server.appmaster.state.RoleInstance;
@@ -31,7 +31,10 @@ import org.apache.slider.server.appmaster.state.RoleStatus;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
 import org.apache.slider.server.appmaster.web.WebAppApi;
 import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource;
-import org.apache.slider.server.appmaster.web.rest.RestPaths;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
+
+import org.apache.slider.server.appmaster.web.rest.application.resources.AggregateModelRefresher;
+import org.apache.slider.server.appmaster.web.rest.application.resources.AppconfRefresher;
 import org.apache.slider.server.appmaster.web.rest.application.resources.CachedContent;
 import org.apache.slider.server.appmaster.web.rest.application.resources.LiveContainersRefresher;
 import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache;
@@ -63,23 +66,46 @@ public class ApplicationResource extends AbstractSliderResource {
       "nodes",
       "statistics",
       "internal");
+
   public static final List<String> ROOT_ENTRIES =
       toJsonList("model", "live", "actions");
+
+  public static final List<String> MODEL_ENTRIES =
+      toJsonList("desired", "resolved");
+
   private final ContentCache cache = new ContentCache();
   private final StateAccessForProviders state;
 
   public ApplicationResource(WebAppApi slider) {
     super(slider);
     state = slider.getAppState();
-    cache.put(RestPaths.LIVE_RESOURCES,
+    cache.put(LIVE_RESOURCES,
         new CachedContent<ConfTree>(LIFESPAN,
             new LiveResourcesRefresher(state)));
-    cache.put(RestPaths.LIVE_CONTAINERS,
+    cache.put(LIVE_CONTAINERS,
         new CachedContent<Map<String, SerializedContainerInformation>>(LIFESPAN,
             new LiveContainersRefresher(state)));
-    cache.put(RestPaths.LIVE_COMPONENTS,
+    cache.put(LIVE_COMPONENTS,
         new CachedContent<Map<String, SerializedComponentInformation>> (LIFESPAN,
             new LiveComponentsRefresher(state)));
+    cache.put(MODEL_DESIRED,
+        new CachedContent<AggregateConf>(LIFESPAN,
+            new AggregateModelRefresher(state, false)));
+    cache.put(MODEL_RESOLVED,
+        new CachedContent<AggregateConf>(LIFESPAN,
+            new AggregateModelRefresher(state, true)));
+    cache.put(MODEL_RESOLVED_APPCONF,
+        new CachedContent<ConfTree>(LIFESPAN,
+            new AppconfRefresher(state, false, false)));
+    cache.put(MODEL_RESOLVED_RESOURCES,
+        new CachedContent<ConfTree>(LIFESPAN,
+            new AppconfRefresher(state, false, true)));
+    cache.put(MODEL_DESIRED_APPCONF,
+        new CachedContent<ConfTree>(LIFESPAN,
+            new AppconfRefresher(state, true, false)));
+    cache.put(MODEL_DESIRED_RESOURCES,
+        new CachedContent<ConfTree>(LIFESPAN,
+            new AppconfRefresher(state, true, true)));
   }
 
   /**
@@ -98,45 +124,87 @@ public class ApplicationResource extends AbstractSliderResource {
     return ROOT_ENTRIES;
   }
 
+  /**
+   * Enum model values: desired and resolved
+   * @return the desired and resolved model
+   */
   @GET
-  @Path("/model")
+  @Path(MODEL)
   @Produces({MediaType.APPLICATION_JSON})
   public List<String> getModel() {
-    return toJsonList("desired", "resolved");
+    return MODEL_ENTRIES;
   }
 
   @GET
-  @Path("/live")
+  @Path(MODEL_DESIRED)
+  @Produces({MediaType.APPLICATION_JSON})
+  public AggregateConf getModelDesired() {
+    return lookupAggregateConf(MODEL_DESIRED);
+  }
+  
+  @GET
+  @Path(MODEL_DESIRED_APPCONF)
+  @Produces({MediaType.APPLICATION_JSON})
+  public ConfTree getModelDesiredAppconf() {
+    return lookupConfTree(MODEL_DESIRED_APPCONF);
+  }
+
+  @GET
+  @Path(MODEL_DESIRED_RESOURCES)
+  @Produces({MediaType.APPLICATION_JSON})
+  public ConfTree getModelDesiredResources() {
+    return lookupConfTree(MODEL_DESIRED_RESOURCES);
+  }
+  
+  @GET
+  @Path(MODEL_RESOLVED)
+  @Produces({MediaType.APPLICATION_JSON})
+  public AggregateConf getModelResolved() {
+    return lookupAggregateConf(MODEL_RESOLVED);
+  }
+
+  @GET
+  @Path(MODEL_RESOLVED_APPCONF)
+  @Produces({MediaType.APPLICATION_JSON})
+  public ConfTree getModelResolvedAppconf() {
+    return lookupConfTree(MODEL_RESOLVED_APPCONF);
+  }
+
+  @GET
+  @Path(MODEL_RESOLVED_RESOURCES)
+  @Produces({MediaType.APPLICATION_JSON})
+  public ConfTree getModelResolvedResources() {
+    return lookupConfTree(MODEL_RESOLVED_RESOURCES);
+  }
+  
+  @GET
+  @Path(LIVE)
   @Produces({MediaType.APPLICATION_JSON})
   public List<String> getLive() {
     return LIVE_ENTRIES;
   }
 
   @GET
-  @Path(RestPaths.LIVE_RESOURCES)
+  @Path(LIVE_RESOURCES)
   @Produces({MediaType.APPLICATION_JSON})
   public Object getLiveResources() {
-    try {
-      return cache.lookup(RestPaths.LIVE_RESOURCES);
-    } catch (Exception e) {
-      throw buildException(RestPaths.LIVE_RESOURCES, e);
-    }
+    return lookupConfTree(LIVE_RESOURCES);
   }
   
   @GET
-  @Path(RestPaths.LIVE_CONTAINERS)
+  @Path(LIVE_CONTAINERS)
   @Produces({MediaType.APPLICATION_JSON})
   public Map<String, SerializedContainerInformation> getLiveContainers() {
     try {
       return (Map<String, SerializedContainerInformation>)cache.lookup(
-          RestPaths.LIVE_CONTAINERS);
+          LIVE_CONTAINERS);
     } catch (Exception e) {
-      throw buildException(RestPaths.LIVE_CONTAINERS, e);
+      throw buildException(LIVE_CONTAINERS, e);
     }
   }
 
   @GET
-  @Path(RestPaths.LIVE_CONTAINERS + "/{containerId}")
+  @Path(LIVE_CONTAINERS + "/{containerId}")
   @Produces({MediaType.APPLICATION_JSON})
   public SerializedContainerInformation getLiveContainer(
       @PathParam("containerId") String containerId) {
@@ -146,24 +214,24 @@ public class ApplicationResource extends AbstractSliderResource {
     } catch (NoSuchNodeException e) {
       throw new NotFoundException("Unknown container: " + containerId);
     } catch (Exception e) {
-      throw buildException(RestPaths.LIVE_CONTAINERS + "/"+ containerId, e);
+      throw buildException(LIVE_CONTAINERS + "/"+ containerId, e);
     }
   }
 
   @GET
-  @Path(RestPaths.LIVE_COMPONENTS)
+  @Path(LIVE_COMPONENTS)
   @Produces({MediaType.APPLICATION_JSON})
   public Map<String, SerializedComponentInformation> getLiveComponents() {
     try {
       return (Map<String, SerializedComponentInformation>) cache.lookup(
-          RestPaths.LIVE_COMPONENTS);
+          LIVE_COMPONENTS);
     } catch (Exception e) {
-      throw buildException(RestPaths.LIVE_COMPONENTS, e);
+      throw buildException(LIVE_COMPONENTS, e);
     }
   }
   
   @GET
-  @Path(RestPaths.LIVE_COMPONENTS+"/{component}")
+  @Path(LIVE_COMPONENTS + "/{component}")
   @Produces({MediaType.APPLICATION_JSON})
   public SerializedComponentInformation getLiveComponent(
       @PathParam("component") String component) {
@@ -179,11 +247,16 @@ public class ApplicationResource extends AbstractSliderResource {
     } catch (YarnRuntimeException e) {
       throw new NotFoundException("Unknown component: " + component);
     } catch (Exception e) {
-      throw buildException(RestPaths.LIVE_CONTAINERS, e);
+      throw buildException(LIVE_CONTAINERS, e);
     }
   }
-  
-  List<RoleInstance> lookupRoleContainers(String component) {
+
+  /**
+   * Look up all containers of a specific component name 
+   * @param component component/role name
+   * @return list of instances. This is a snapshot
+   */
+  private List<RoleInstance> lookupRoleContainers(String component) {
     RoleStatus roleStatus = state.lookupRoleStatus(component);
     List<RoleInstance> ownedContainerList = state.cloneOwnedContainerList();
     List<RoleInstance> matching = new ArrayList<RoleInstance>(ownedContainerList.size());
@@ -195,5 +268,21 @@ public class ApplicationResource extends AbstractSliderResource {
     }
     return matching;
   }
-  
+
+  protected AggregateConf lookupAggregateConf(String key) {
+    try {
+      return (AggregateConf) cache.lookup(key);
+    } catch (Exception e) {
+      throw buildException(key, e);
+    }
+  }
+
+  protected ConfTree lookupConfTree(String key) {
+    try {
+      return (ConfTree) cache.lookup(key);
+    } catch (Exception e) {
+      throw buildException(key, e);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AggregateModelRefresher.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AggregateModelRefresher.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AggregateModelRefresher.java
new file mode 100644
index 0000000..ee28abf
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AggregateModelRefresher.java
@@ -0,0 +1,47 @@
+/*
+ * 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.web.rest.application.resources;
+
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+
+/**
+ * Refresh the aggregate desired model via
+ * {@link StateAccessForProviders#getInstanceDefinitionSnapshot()}
+ */
+public class AggregateModelRefresher
+    implements ResourceRefresher<AggregateConf> {
+
+  private final StateAccessForProviders state;
+  private final boolean resolved;
+
+  public AggregateModelRefresher(StateAccessForProviders state,
+      boolean resolved) {
+    this.state = state;
+    this.resolved = resolved;
+  }
+
+  @Override
+  public AggregateConf refresh() throws Exception {
+    return
+        resolved ?
+          state.getInstanceDefinitionSnapshot()
+          : state.getUnresolvedInstanceDefinition();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AppconfRefresher.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AppconfRefresher.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AppconfRefresher.java
new file mode 100644
index 0000000..65582ed
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AppconfRefresher.java
@@ -0,0 +1,55 @@
+/*
+ * 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.web.rest.application.resources;
+
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.persist.ConfTreeSerDeser;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+
+/**
+ * refresher for resources and application configuration
+ */
+public class AppconfRefresher
+    implements ResourceRefresher<ConfTree> {
+
+  private final StateAccessForProviders state;
+  private final boolean unresolved;
+  private final boolean resources;
+
+  public AppconfRefresher(StateAccessForProviders state,
+      boolean unresolved,
+      boolean resources) {
+    this.state = state;
+    this.unresolved = unresolved;
+    this.resources = resources;
+  }
+
+
+  @Override
+  public ConfTree refresh() throws Exception {
+    AggregateConf aggregateConf =
+        unresolved ?
+        state.getUnresolvedInstanceDefinition():
+        state.getInstanceDefinitionSnapshot();
+    ConfTree ct = resources ? aggregateConf.getResources() 
+                            : aggregateConf.getResources();
+    return new ConfTreeSerDeser().fromInstance(ct);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java
index 78e65e8..22fd0fe 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java
@@ -49,7 +49,7 @@ public class CachedContent<T> {
    * Get the value, triggering a refresh if needed
    * @return the cached or latest value.
    */
-  public T get() {
+  public T get() throws Exception {
     maybeRefresh();
     return getCachedValue();
   }
@@ -78,7 +78,7 @@ public class CachedContent<T> {
    * Maybe refresh the content
    * @return true if a refresh took place.
    */
-  public synchronized boolean maybeRefresh() {
+  public synchronized boolean maybeRefresh() throws Exception {
     long now = now();
     if (cachedValue == null || now >= expires) {
       log.debug("Refreshing at time {}", now);
@@ -97,7 +97,7 @@ public class CachedContent<T> {
    * Force a refresh and reset the expiry counter
    * @return the new value
    */
-  protected synchronized T forceRefresh() {
+  protected synchronized T forceRefresh() throws Exception {
     refreshCounter ++;
     T updated = refresh.refresh();
     Preconditions.checkNotNull(updated);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java
index 6ee9604..b60e9b3 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java
@@ -19,7 +19,6 @@
 package org.apache.slider.server.appmaster.web.rest.application.resources;
 
 import com.google.common.base.Preconditions;
-import org.apache.slider.server.appmaster.web.rest.RestPaths;
 
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -35,7 +34,7 @@ public class ContentCache extends ConcurrentHashMap<String, CachedContent> {
   public ContentCache() {
   }
   
-  public Object lookup(String key) {
+  public Object lookup(String key) throws Exception {
     CachedContent content = get(key);
     Preconditions.checkNotNull(content, "no content for path " + key);
     return content.get();

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveComponentsRefresher.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveComponentsRefresher.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveComponentsRefresher.java
index e543265..530bbd3 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveComponentsRefresher.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveComponentsRefresher.java
@@ -35,7 +35,8 @@ public class LiveComponentsRefresher
   }
 
   @Override
-  public Map<String, SerializedComponentInformation> refresh() {
+  public Map<String, SerializedComponentInformation> refresh() throws
+      Exception {
 
     Map<Integer, RoleStatus> roleStatusMap = state.getRoleStatusMap();
     Map<String, SerializedComponentInformation> results =

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveContainersRefresher.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveContainersRefresher.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveContainersRefresher.java
index 39a543b..1b29801 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveContainersRefresher.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveContainersRefresher.java
@@ -38,7 +38,8 @@ public class LiveContainersRefresher implements ResourceRefresher<Map<String, Se
   }
 
   @Override
-  public Map<String, SerializedContainerInformation> refresh() {
+  public Map<String, SerializedContainerInformation> refresh() throws
+      Exception {
     List<RoleInstance> containerList = state.cloneOwnedContainerList();
 
     Map<String, SerializedContainerInformation> map = new HashMap<String, SerializedContainerInformation>();

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java
index dd845d8..f988297 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java
@@ -35,7 +35,7 @@ public class LiveResourcesRefresher implements ResourceRefresher<ConfTree> {
   }
 
   @Override
-  public ConfTree refresh() {
+  public ConfTree refresh() throws Exception {
 
     // snapshot resources
     ConfTreeOperations resources = state.getResourcesSnapshot();

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java
index 9822fbc..35f0367 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java
@@ -27,5 +27,5 @@ public interface ResourceRefresher<T> {
    * Build an up to date version of the data
    * @return a new instance of the (JSON serializable) data
    */
-  T refresh();
+  T refresh() throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceSnapshotRefresher.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceSnapshotRefresher.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceSnapshotRefresher.java
index 0034d57..c16912a 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceSnapshotRefresher.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceSnapshotRefresher.java
@@ -31,7 +31,7 @@ public class ResourceSnapshotRefresher implements ResourceRefresher<ConfTree> {
   }
 
   @Override
-  public ConfTree refresh() {
+  public ConfTree refresh() throws Exception {
 
     // snapshot resources
     ConfTreeOperations resources = state.getResourcesSnapshot();

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentWeb.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentWeb.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentWeb.groovy
index 2ad7129..6df0452 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentWeb.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentWeb.groovy
@@ -26,6 +26,9 @@ import org.apache.slider.agent.AgentMiniClusterTestBase
 import org.apache.slider.api.StateValues
 import org.apache.slider.api.types.SerializedComponentInformation
 import org.apache.slider.api.types.SerializedContainerInformation
+import org.apache.slider.common.params.Arguments
+import org.apache.slider.core.conf.AggregateConf
+import org.apache.slider.core.conf.ConfTree
 import org.apache.slider.server.appmaster.web.rest.application.ApplicationResource
 
 import static org.apache.slider.api.ResourceKeys.*
@@ -45,6 +48,8 @@ import static org.apache.slider.server.appmaster.management.MetricsKeys.*
 class TestStandaloneAgentWeb extends AgentMiniClusterTestBase {
   
   public static final int WEB_STARTUP_TIME = 30000
+  public static final String TEST_GLOBAL_OPTION = "test.global.option"
+  public static final String TEST_GLOBAL_OPTION_PRESENT = "present"
 
   @Test
   public void testStandaloneAgentWeb() throws Throwable {
@@ -58,7 +63,10 @@ class TestStandaloneAgentWeb extends AgentMiniClusterTestBase {
 
 
     ServiceLauncher<SliderClient> launcher =
-        createStandaloneAM(clustername, true, false)
+        createStandaloneAMWithArgs(clustername,
+            [Arguments.ARG_OPTION,
+             TEST_GLOBAL_OPTION, TEST_GLOBAL_OPTION_PRESENT],
+            true, false)
     SliderClient client = launcher.service
     addToTeardown(client);
 
@@ -139,11 +147,8 @@ class TestStandaloneAgentWeb extends AgentMiniClusterTestBase {
     assert amContainerInfo.state == StateValues.STATE_LIVE
    
     describe "base entry lists"
-    def list = fetchType(ArrayList, appmaster, LIVE)
 
-    def live_entries = ApplicationResource.LIVE_ENTRIES
-    assert list.size() == live_entries.size()
-    live_entries.containsAll(list)
+    assertPathServesList(appmaster, LIVE, ApplicationResource.LIVE_ENTRIES)
     
     describe "containers"
 
@@ -180,6 +185,55 @@ class TestStandaloneAgentWeb extends AgentMiniClusterTestBase {
     assert amFullInfo.containers.size() == 1
     assert amFullInfo.containers[0] == amContainerId
 
+    testRESTModel(appmaster)
+    
+    
+  }
+
+  public void testRESTModel(String appmaster) {
+    describe "model"
+
+    assertPathServesList(appmaster,
+        MODEL,
+        ApplicationResource.MODEL_ENTRIES)
+
+    def unresolvedConf = fetchType(AggregateConf, appmaster, MODEL_DESIRED)
+    log.info "Unresolved \n$unresolvedConf"
+    def unresolvedAppConf = unresolvedConf.appConfOperations
+
+    def sam = "slider-appmaster"
+    assert unresolvedAppConf.getComponentOpt(sam,
+        TEST_GLOBAL_OPTION, "") == ""
+    def resolvedConf = fetchType(AggregateConf, appmaster, MODEL_RESOLVED)
+    log.info "Resolved \n$resolvedConf"
+    assert resolvedConf.appConfOperations.getComponentOpt(
+        sam, TEST_GLOBAL_OPTION, "") == TEST_GLOBAL_OPTION_PRESENT
+
+    def unresolved = fetchTypeList(ConfTree, appmaster,
+        [MODEL_DESIRED_APPCONF, MODEL_DESIRED_RESOURCES])
+    assert unresolved[0].components[sam][TEST_GLOBAL_OPTION] == null
+
+
+    def resolved = fetchTypeList(ConfTree, appmaster,
+        [MODEL_RESOLVED_APPCONF, MODEL_RESOLVED_RESOURCES])
+    assert resolved[0].components[sam][TEST_GLOBAL_OPTION] ==
+           TEST_GLOBAL_OPTION_PRESENT
+  }
+
+  /**
+   * Assert that a path resolves to an array list that contains
+   * those entries (and only those entries) expected
+   * @param appmaster AM ref
+   * @param path path under AM
+   * @param entries entries to assert the presence of
+   */
+  public void assertPathServesList(
+      String appmaster,
+      String path,
+      List<String> entries) {
+    def list = fetchType(ArrayList, appmaster, path)
+    assert list.size() == entries.size()
+    assert entries.containsAll(list)
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAppRestIntegration.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAppRestIntegration.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAppRestIntegration.groovy
index 712445d..cf32f33 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAppRestIntegration.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAppRestIntegration.groovy
@@ -128,7 +128,7 @@ class TestMockAppStateAppRestIntegration extends BaseMockAppStateTest implements
   class IntRefresher implements ResourceRefresher<Integer>   {
     int count ;
     @Override
-    Integer refresh() {
+    Integer refresh() throws Exception {
       log.info("Refresh at $count")
       def result = count
       count += 1;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d2a35bb/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 f9857a8..806285b 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
@@ -1184,4 +1184,21 @@ class SliderTestUtils extends Assert {
     ConfTreeOperations tree = new ConfTreeOperations(ctree)
     return tree
   }
+
+  /**
+   * Fetch a list of URLs, all of which must be of the same type
+   * @param clazz class of resolved values
+   * @param appmaster URL to app master
+   * @param subpaths list of subpaths
+   * @return a list of values in the same order as the paths passed in
+   */
+  public <T> List<T> fetchTypeList(
+      Class<T> clazz, String appmaster, List<String> subpaths
+      ) {
+    List<T> results = []
+    subpaths.each { String it ->
+      results.add(fetchType(clazz, appmaster, it))
+    }
+    return results;
+  }
 }