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/12/18 22:32:49 UTC

[41/50] incubator-slider git commit: SLIDER-711 live/resources fetch working, with test

SLIDER-711 live/resources fetch working, with test


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

Branch: refs/heads/feature/SLIDER-151_REST_API
Commit: ca4686de3c21ee0565c2fbab58943838e7654c97
Parents: 8e18403
Author: Steve Loughran <st...@apache.org>
Authored: Wed Dec 17 13:03:21 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Dec 17 13:03:21 2014 +0000

----------------------------------------------------------------------
 .../java/org/apache/slider/api/StatusKeys.java  |  2 +-
 .../org/apache/slider/client/SliderClient.java  |  3 +-
 .../org/apache/slider/common/SliderKeys.java    |  6 +-
 .../apache/slider/core/build/InstanceIO.java    | 14 ++--
 .../server/appmaster/SliderAppMaster.java       |  1 +
 .../web/rest/AbstractSliderResource.java        | 30 ++++++++
 .../server/appmaster/web/rest/RestPaths.java    |  3 +-
 .../rest/application/ApplicationResource.java   | 22 ++----
 .../application/resources/CachedContent.java    | 10 ++-
 .../application/resources/ContentCache.java     |  9 ++-
 .../resources/LiveResourcesRefresher.java       |  2 +-
 .../web/rest/registry/RegistryResource.java     | 18 ++---
 .../standalone/TestStandaloneAgentWeb.groovy    | 46 +++++++++++--
 ...estMockAppStateAppResourceIntegration.groovy | 72 ++++++++++++++++++++
 14 files changed, 185 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ca4686de/slider-core/src/main/java/org/apache/slider/api/StatusKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/api/StatusKeys.java b/slider-core/src/main/java/org/apache/slider/api/StatusKeys.java
index 92cc8aa..ef68aad 100644
--- a/slider-core/src/main/java/org/apache/slider/api/StatusKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/api/StatusKeys.java
@@ -82,7 +82,7 @@ public interface StatusKeys {
    * info: #of instances of a component requested: {@value}
    *
    */
-  String COMPONENT_INSTANCES_REQUESTED = COMPONENT_INSTANCES + ".requested";
+  String COMPONENT_INSTANCES_REQUESTING = COMPONENT_INSTANCES + ".requesting";
 
   /**
    * info: #of instances of a component being released: {@value}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ca4686de/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index 813ae0f..f9aae6f 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -2296,7 +2296,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     int exitCode = EXIT_FALSE;
     // save the specification
     try {
-      InstanceIO.updateInstanceDefinition(sliderFileSystem, clusterDirectory, instanceDefinition);
+      InstanceIO.saveInstanceDefinition(sliderFileSystem, clusterDirectory,
+          instanceDefinition);
     } catch (LockAcquireFailedException e) {
       // lock failure
       log.debug("Failed to lock dir {}", clusterDirectory, e);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ca4686de/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java b/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
index b9dc8e1..701ef14 100644
--- a/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
@@ -24,11 +24,13 @@ package org.apache.slider.common;
  */
 public interface SliderKeys extends SliderXmlConfKeys {
 
-  
+  /**
+   * This is the name of the slider appmaster in configurations :{@value}
+   */
   String COMPONENT_AM = "slider-appmaster";
   
   /**
-   * Slider role is "special"
+   * Slider role is "special":{@value}
    */
   int ROLE_AM_PRIORITY_INDEX = 0;
   

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ca4686de/slider-core/src/main/java/org/apache/slider/core/build/InstanceIO.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/build/InstanceIO.java b/slider-core/src/main/java/org/apache/slider/core/build/InstanceIO.java
index 25bb4ab..3a8f805 100644
--- a/slider-core/src/main/java/org/apache/slider/core/build/InstanceIO.java
+++ b/slider-core/src/main/java/org/apache/slider/core/build/InstanceIO.java
@@ -64,17 +64,17 @@ public class InstanceIO {
 
 
   /**
-   * Update an instance definition
-   * @param coreFS
-   * @param dir
-   * @param instanceDefinition
+   * Update a persisted instance definition
+   * @param coreFS filesystem
+   * @param dir directory to load from
+   * @param instanceDefinition instance definition to save do
    * @throws SliderException
    * @throws IOException
    * @throws LockAcquireFailedException
    */
-  public static void updateInstanceDefinition(CoreFileSystem coreFS,
-                                              Path dir,
-                                              AggregateConf instanceDefinition)
+  public static void saveInstanceDefinition(CoreFileSystem coreFS,
+      Path dir,
+      AggregateConf instanceDefinition)
       throws SliderException, IOException, LockAcquireFailedException {
     ConfPersister persister =
       new ConfPersister(coreFS, dir);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ca4686de/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 67e050a..c8764c9 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
@@ -933,6 +933,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
         RestPaths.WS_CONTEXT)
            .withHttpPolicy(serviceConf, HttpConfig.Policy.HTTP_ONLY)
            .at(port)
+           .inDevMode()
            .start(webApp);
 
     WebAppService<SliderAMWebApp> webAppService =

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ca4686de/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java
index 90fe6fe..ede46f0 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java
@@ -18,6 +18,11 @@
 
 package org.apache.slider.server.appmaster.web.rest;
 
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.registry.client.exceptions.AuthenticationFailedException;
+import org.apache.hadoop.registry.client.exceptions.NoPathPermissionsException;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
+import org.apache.hadoop.yarn.webapp.NotFoundException;
 import org.apache.slider.server.appmaster.web.WebAppApi;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -60,4 +65,29 @@ public abstract class AbstractSliderResource {
     }
   }
 
+  /**
+   * Convert any exception caught into a web application
+   * exception for rethrowing
+   * @param path path of request
+   * @param ex exception
+   * @return an exception to throw
+   */
+  public WebApplicationException buildException(String path,
+      Exception ex) {
+    try {
+      throw ex;
+    } catch (WebApplicationException e) {
+      // rethrow direct
+      throw e;
+    } catch (PathNotFoundException e) {
+      return new NotFoundException("Not found: " + path);
+    } catch (AuthenticationFailedException e) {
+      return new ForbiddenException(path);
+    } catch (NoPathPermissionsException e) {
+      return new ForbiddenException(path);
+    } catch (Exception e) {
+      log.error("Error during generation of response: {}", e, e);
+      return new WebApplicationException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ca4686de/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 bd569ef..5dbc090 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
@@ -121,8 +121,7 @@ public class RestPaths {
    * application subpath
    */
   public static final String SLIDER_SUBPATH_APPLICATION = "/application";
-
-
+  
   /**
    * management path: {@value}
    */

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ca4686de/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 2477464..b84717d 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
@@ -29,12 +29,14 @@ import org.apache.slider.server.appmaster.web.rest.application.resources.LiveRes
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.inject.Singleton;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 import java.util.List;
 
+@Singleton
 public class ApplicationResource extends AbstractSliderResource {
   private static final Logger log =
       LoggerFactory.getLogger(ApplicationResource.class);
@@ -73,13 +75,6 @@ public class ApplicationResource extends AbstractSliderResource {
   }
 
   @GET
-  @Path("/model/")
-  @Produces({MediaType.APPLICATION_JSON})
-  public List<String> getModelSlash() {
-    return getModel();
-  }
-
-  @GET
   @Path("/live")
   @Produces({MediaType.APPLICATION_JSON})
   public List<String> getLive() {
@@ -92,17 +87,14 @@ public class ApplicationResource extends AbstractSliderResource {
   }
 
   @GET
-  @Path("/live/")
-  @Produces({MediaType.APPLICATION_JSON})
-  public List<String> getLiveSlash() {
-    return getLive();
-  }
-
-  @GET
   @Path(RestPaths.LIVE_RESOURCES)
   @Produces({MediaType.APPLICATION_JSON})
   public Object getLiveResources() {
-    return cache.get(RestPaths.LIVE_RESOURCES).get();
+    try {
+      return cache.get(RestPaths.LIVE_RESOURCES).get();
+    } catch (Exception e) {
+      throw buildException(RestPaths.LIVE_RESOURCES, e);
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ca4686de/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 c2edd61..c11a3ee 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
@@ -69,14 +69,18 @@ public class CachedContent<T> {
    * @return true if a refresh took place.
    */
   public boolean maybeRefresh() {
-    long now = Time.monotonicNow();
-    if (now > expires) {
+    long now = now();
+    if (cachedValue == null || now > expires) {
       forceRefresh();
       return true;
     }
     return false;
   }
 
+  protected long now() {
+    return Time.monotonicNow();
+  }
+
   /**
    * Force a refresh and reset the expiry counter
    * @return the new value
@@ -85,7 +89,7 @@ public class CachedContent<T> {
     T updated = refresh.refresh();
     Preconditions.checkNotNull(updated);
     cachedValue = updated;
-    expires = Time.monotonicNow() + lifespan;
+    expires = now() + lifespan;
     return updated;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ca4686de/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 0cc9eb4..169eaa3 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
@@ -21,6 +21,11 @@ package org.apache.slider.server.appmaster.web.rest.application.resources;
 import java.util.HashMap;
 
 public class ContentCache extends HashMap<String, CachedContent> {
-  
-  
+
+  public ContentCache(int initialCapacity) {
+    super(initialCapacity);
+  }
+
+  public ContentCache() {
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ca4686de/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 fd52528..dd845d8 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
@@ -45,7 +45,7 @@ public class LiveResourcesRefresher implements ResourceRefresher<ConfTree> {
     for (RoleStatus status : roleStatusMap.values()) {
       String name = status.getName();
       resources.setComponentOpt(name,
-          StatusKeys.COMPONENT_INSTANCES_REQUESTED,
+          StatusKeys.COMPONENT_INSTANCES_REQUESTING,
           status.getRequested());
       resources.setComponentOpt(name,
           StatusKeys.COMPONENT_INSTANCES_ACTUAL,

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ca4686de/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java
index 70c0826..c824848 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.registry.client.exceptions.NoRecordException;
 import org.apache.hadoop.yarn.webapp.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,7 +54,7 @@ import java.io.IOException;
  * 
  */
 @Singleton
-public class RegistryResource {
+public class RegistryResource extends AbstractSliderResource {
   protected static final Logger log =
       LoggerFactory.getLogger(RegistryResource.class);
   public static final String SERVICE_PATH =
@@ -66,9 +67,11 @@ public class RegistryResource {
    * @param slider slider API
    */
   public RegistryResource(WebAppApi slider) {
+    super(slider);
     this.registry = slider.getRegistryOperations();
   }
 
+  
   /**
    * Internal init code, per request
    * @param request incoming request 
@@ -114,18 +117,9 @@ public class RegistryResource {
         log.debug("Resolved:\n{}", pathEntry);
       }
       return pathEntry;
-    } catch (WebApplicationException e) {
-      // rethrow direct
-      throw e;
-    } catch (PathNotFoundException e) {
-      throw new NotFoundException("Not found: " + path);
-    } catch (AuthenticationFailedException e) {
-      throw new ForbiddenException(path);
-    } catch (NoPathPermissionsException e) {
-      throw new ForbiddenException(path);
+   
     } catch (Exception e) {
-      log.error("Error during generation of response: {}", e, e);
-      throw new WebApplicationException(e);
+      throw buildException(path, e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ca4686de/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 577ad73..d524698 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
@@ -21,10 +21,16 @@ package org.apache.slider.agent.standalone
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.yarn.api.records.ApplicationReport
+import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.slider.agent.AgentMiniClusterTestBase
+import static org.apache.slider.api.ResourceKeys.*
+import static org.apache.slider.api.StatusKeys.*
 import org.apache.slider.client.SliderClient
+import static org.apache.slider.common.SliderKeys.*;
+import org.apache.slider.core.conf.ConfTreeOperations
 import org.apache.slider.core.main.ServiceLauncher
-import org.apache.slider.server.appmaster.web.rest.RestPaths
+import org.apache.slider.core.persist.ConfTreeSerDeser
+
 import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
 import org.junit.Test
 
@@ -33,8 +39,7 @@ import static org.apache.slider.server.appmaster.management.MetricsKeys.*
 @CompileStatic
 @Slf4j
 class TestStandaloneAgentWeb extends AgentMiniClusterTestBase {
-
-  
+ 
   @Test
   public void testStandaloneAgentWeb() throws Throwable {
 
@@ -55,9 +60,11 @@ class TestStandaloneAgentWeb extends AgentMiniClusterTestBase {
     def realappmaster = report.originalTrackingUrl
     execHttpRequest(30000) {
       GET(realappmaster)
-    } 
-    def metrics = GET(realappmaster, SYSTEM_METRICS)
-    log.info metrics
+    }
+    execHttpRequest(30000) {
+      def metrics = GET(realappmaster, SYSTEM_METRICS)
+      log.info metrics
+    }
     
     sleep(5000)
     def appmaster = report.trackingUrl
@@ -82,9 +89,34 @@ class TestStandaloneAgentWeb extends AgentMiniClusterTestBase {
     // now some REST gets
     describe "Application REST GETs"
 
-    getWebPage(conf, appmaster, SLIDER_PATH_APPLICATION + LIVE_RESOURCES)
+    ConfTreeOperations tree = fetchConfigTree(conf, appmaster, LIVE_RESOURCES)
 
+    log.info tree.toString()
+    def liveAM = tree.getComponent(COMPONENT_AM)
+    def desiredInstances = liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES);
+    assert desiredInstances == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_ACTUAL)
+
+    assert 1 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_STARTED)
+    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_REQUESTING)
+    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_FAILED)
+    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_COMPLETED)
+    assert 0 == liveAM.getMandatoryOptionInt(COMPONENT_INSTANCES_RELEASING)
+
+  }
 
+  public ConfTreeOperations fetchConfigTree(
+      YarnConfiguration conf,
+      String appmaster,
+      String subpath) {
+    ConfTreeSerDeser serDeser = new ConfTreeSerDeser()
+
+    def json = getWebPage(
+        conf,
+        appmaster,
+        SLIDER_PATH_APPLICATION + subpath)
+    def ctree = serDeser.fromJson(json)
+    ConfTreeOperations tree = new ConfTreeOperations(ctree)
+    return tree
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ca4686de/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAppResourceIntegration.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAppResourceIntegration.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAppResourceIntegration.groovy
new file mode 100644
index 0000000..2d0b8bf
--- /dev/null
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAppResourceIntegration.groovy
@@ -0,0 +1,72 @@
+/*
+ * 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.model.appstate
+
+import groovy.util.logging.Slf4j
+import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
+import org.apache.slider.server.appmaster.model.mock.MockRoles
+import org.apache.slider.server.appmaster.web.rest.application.resources.CachedContent
+import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache
+import org.apache.slider.server.appmaster.web.rest.application.resources.ResourceRefresher
+import org.junit.Test
+
+@Slf4j
+class TestMockAppStateAppResourceIntegration extends BaseMockAppStateTest implements MockRoles {
+
+  @Override
+  String getTestName() {
+    return "TestMockAppStateAppResourceIntegration"
+  }
+
+  @Test
+  public void testCachedDocument() throws Throwable {
+    ContentCache cache = new ContentCache()
+
+    def content = new CachedContentManagedTimer(new IntRefresher())
+    cache.put("/int", content)
+    def content1 = cache.get("/int")
+    assert content.equals(content1)
+    
+    assert 0 == content.get()
+    assert 0 == content.getCachedValue()
+    
+  }
+
+  class IntRefresher implements ResourceRefresher<Integer>   {
+    int count ;
+    @Override
+    Integer refresh() {
+      return count++;
+    }
+  }
+
+  class CachedContentManagedTimer extends CachedContent {
+    int time = 0;
+        
+    @Override
+    protected long now() {
+      return time++;
+    }
+
+    CachedContentManagedTimer(ResourceRefresher refresh) {
+      super(1, refresh)
+    }
+    
+  }
+}