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/11/17 22:12:53 UTC

[3/8] incubator-slider git commit: SLIDER-979 AM web UI to show state of AA request

SLIDER-979 AM web UI to show state of AA request


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: 8efc4c2c5e7af89ffd36af25159c1ac5afad2759
Parents: c776b1a
Author: Steve Loughran <st...@apache.org>
Authored: Mon Nov 16 16:12:10 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Nov 16 16:12:10 2015 +0000

----------------------------------------------------------------------
 .../server/appmaster/management/LongGauge.java  |  6 ++
 .../slider/server/appmaster/state/AppState.java |  3 +-
 .../server/appmaster/web/view/IndexBlock.java   |  9 ++-
 .../providers/agent/DemoAgentAAEcho.groovy      | 45 +++++++++++
 .../providers/agent/TestAgentAAEcho.groovy      |  9 ++-
 .../appstate/BaseMockAppStateAATest.groovy      | 62 ++++++++++++++++
 .../appstate/TestMockAppStateAAPlacement.groovy | 41 +---------
 .../appstate/TestMockLabelledAAPlacement.groovy | 55 ++++----------
 .../appmaster/model/mock/MockFactory.groovy     | 37 +++++++++-
 .../appmaster/model/mock/MockRoles.groovy       |  2 +
 .../appmaster/web/view/TestIndexBlock.groovy    | 78 +++++++++++++++-----
 .../slider/server/management/TestGauges.groovy  | 52 +++++++++++++
 .../apache/slider/test/SliderTestUtils.groovy   | 45 ++++++++---
 13 files changed, 324 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8efc4c2c/slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongGauge.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongGauge.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongGauge.java
index ac9ac0e..c93467b 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongGauge.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongGauge.java
@@ -56,6 +56,11 @@ public class LongGauge extends AtomicLong implements Metric, Gauge<Long> {
     return get();
   }
 
+  /**
+   * Method from {@Code counter}; used here for drop-in replacement
+   * without any recompile
+   * @return current value
+   */
   public Long getCount() {
     return get();
   }
@@ -66,6 +71,7 @@ public class LongGauge extends AtomicLong implements Metric, Gauge<Long> {
   public void inc() {
     incrementAndGet();
   }
+
   /**
    * {@code --}
    */

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8efc4c2c/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 d977323..4152a89 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
@@ -1231,13 +1231,14 @@ public class AppState {
    * @return the container request to submit or null if there is none
    */
   private AMRMClient.ContainerRequest createContainerRequest(RoleStatus role) {
-    incrementRequestCount(role);
     if (role.isAntiAffinePlacement()) {
       return createAAContainerRequest(role);
     } else {
+      incrementRequestCount(role);
       return roleHistory.requestContainerForRole(role).getIssuedRequest();
     }
   }
+
   /**
    * Create a container request.
    * Update internal state, such as the role request count.

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8efc4c2c/slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java
index 5131b5e..41e1c01 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java
@@ -45,6 +45,13 @@ import java.util.Map.Entry;
 public class IndexBlock extends HtmlBlock {
   private static final Logger log = LoggerFactory.getLogger(IndexBlock.class);
 
+  /**
+   * Message printed when application is at full size.
+   *
+   * {@value}
+   */
+  public static final String ALL_CONTAINERS_ALLOCATED = "all containers allocated";
+
   private StateAccessForProviders appView;
   private ProviderService providerService;
 
@@ -77,7 +84,7 @@ public class IndexBlock extends HtmlBlock {
         appView.getApplicationLivenessInformation();
     String livestatus =
         liveness.allRequestsSatisfied
-        ? "all containers allocated"
+        ? ALL_CONTAINERS_ALLOCATED
         : String.format("Awaiting %d containers", liveness.requestsOutstanding);
     Hamlet.TABLE<DIV<Hamlet>> table1 = div.table();
     table1.tr()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8efc4c2c/slider-core/src/test/groovy/org/apache/slider/providers/agent/DemoAgentAAEcho.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/providers/agent/DemoAgentAAEcho.groovy b/slider-core/src/test/groovy/org/apache/slider/providers/agent/DemoAgentAAEcho.groovy
new file mode 100644
index 0000000..6f21006
--- /dev/null
+++ b/slider-core/src/test/groovy/org/apache/slider/providers/agent/DemoAgentAAEcho.groovy
@@ -0,0 +1,45 @@
+/*
+ * 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.providers.agent
+
+import org.apache.hadoop.yarn.api.records.ApplicationReport
+import org.apache.slider.client.SliderClient
+
+/**
+ * Overridde the test actions with a sleep command, so that developers
+ * can see what the application is up to
+ */
+class DemoAgentAAEcho extends TestAgentAAEcho {
+
+  @Override
+  protected void postLaunchActions(
+      SliderClient sliderClient,
+      String clustername,
+      String roleName,
+      Map<String, Integer> roles) {
+
+    def applicationReport = sliderClient.applicationReport
+    def url = applicationReport.trackingUrl
+    // spin repeating the URl in the logs so YARN chatter doesn't lose it
+    1..5.each {
+      describe("Web UI is at $url")
+      sleep(60 *1000)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8efc4c2c/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAAEcho.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAAEcho.groovy b/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAAEcho.groovy
index 80ff5a8..3835330 100644
--- a/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAAEcho.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAAEcho.groovy
@@ -83,7 +83,7 @@ class TestAgentAAEcho extends TestAgentEcho {
    */
   protected Map<String, Integer> buildRoleMap(String roleName) {
     [
-        (roleName): 2,
+        (roleName): 3,
     ];
   }
 
@@ -108,12 +108,13 @@ class TestAgentAAEcho extends TestAgentEcho {
     // flex size
     // while running, ask for many more, expect them to still be outstanding
     sleep(5000)
+    sliderClient.flex(clustername, [(roleName): 50]);
     waitForRoleCount(sliderClient, onlyOneEcho, 1000)
-    sliderClient.flex(clustername, onlyOneEcho);
 
-    // while running, flex it with no changes
-    sliderClient.flex(clustername, [(roleName): 3]);
+    // while running, flex it to size = 1
     sleep(1000)
+    sliderClient.flex(clustername, onlyOneEcho);
     waitForRoleCount(sliderClient, onlyOneEcho, 1000)
+
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8efc4c2c/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/BaseMockAppStateAATest.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/BaseMockAppStateAATest.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/BaseMockAppStateAATest.groovy
new file mode 100644
index 0000000..7b3a65d
--- /dev/null
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/BaseMockAppStateAATest.groovy
@@ -0,0 +1,62 @@
+/*
+ * 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.transform.CompileStatic
+import groovy.util.logging.Slf4j
+import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
+import org.apache.slider.server.appmaster.model.mock.MockFactory
+import org.apache.slider.server.appmaster.model.mock.MockRoles
+import org.apache.slider.server.appmaster.state.AppStateBindingInfo
+import org.apache.slider.server.appmaster.state.RoleStatus
+
+/**
+ * class for basis of Anti-affine placement tests; sets up role2
+ * for anti-affinity
+ */
+@CompileStatic
+@Slf4j
+class BaseMockAppStateAATest extends BaseMockAppStateTest
+    implements MockRoles {
+
+  /** Role status for the base AA role */
+  RoleStatus aaRole
+
+  /** Role status for the AA role requiring a node with the gpu label */
+  RoleStatus gpuRole
+
+  @Override
+  AppStateBindingInfo buildBindingInfo() {
+    def bindingInfo = super.buildBindingInfo()
+    bindingInfo.roles = [
+        MockFactory.PROVIDER_ROLE0,
+        MockFactory.AAROLE_1_GPU,
+        MockFactory.AAROLE_2,
+    ]
+    bindingInfo
+  }
+
+  @Override
+  void setup() {
+    super.setup()
+    aaRole = lookupRole(MockFactory.AAROLE_2.name)
+    gpuRole = lookupRole(MockFactory.AAROLE_1_GPU.name)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8efc4c2c/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAAPlacement.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAAPlacement.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAAPlacement.groovy
index 01ca2f1..749e4fc 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAAPlacement.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAAPlacement.groovy
@@ -21,23 +21,15 @@ package org.apache.slider.server.appmaster.model.appstate
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.yarn.api.records.Container
-import org.apache.hadoop.yarn.api.records.NodeReport
 import org.apache.hadoop.yarn.api.records.NodeState
 import org.apache.hadoop.yarn.client.api.AMRMClient
-import org.apache.slider.providers.PlacementPolicy
-import org.apache.slider.providers.ProviderRole
-import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
-import org.apache.slider.server.appmaster.model.mock.MockFactory
 import org.apache.slider.server.appmaster.model.mock.MockNodeReport
 import org.apache.slider.server.appmaster.model.mock.MockRoles
 import org.apache.slider.server.appmaster.model.mock.MockYarnEngine
 import org.apache.slider.server.appmaster.operations.AbstractRMOperation
 import org.apache.slider.server.appmaster.state.AppState
-import org.apache.slider.server.appmaster.state.AppStateBindingInfo
 import org.apache.slider.server.appmaster.state.ContainerAssignment
-import org.apache.slider.server.appmaster.state.NodeMap
 import org.apache.slider.server.appmaster.state.RoleInstance
-import org.apache.slider.server.appmaster.state.RoleStatus
 import org.junit.Test
 
 /**
@@ -45,41 +37,12 @@ import org.junit.Test
  */
 @CompileStatic
 @Slf4j
-class TestMockAppStateAAPlacement extends BaseMockAppStateTest
+class TestMockAppStateAAPlacement extends BaseMockAppStateAATest
     implements MockRoles {
 
-  /**
-   * Patch up a "role2" role to have anti-affinity set
-   */
-  public static final ProviderRole AAROLE = new ProviderRole(
-      MockRoles.ROLE2,
-      2,
-      PlacementPolicy.ANTI_AFFINITY_REQUIRED,
-      2,
-      2,
-      null)
-
-  RoleStatus aaRole
   private int NODES = 3
 
   @Override
-  AppStateBindingInfo buildBindingInfo() {
-    def bindingInfo = super.buildBindingInfo()
-    bindingInfo.roles = [
-        MockFactory.PROVIDER_ROLE0,
-        MockFactory.PROVIDER_ROLE1,
-        AAROLE,
-    ]
-    bindingInfo
-  }
-
-  @Override
-  void setup() {
-    super.setup()
-    aaRole = lookupRole(AAROLE.name)
-  }
-
-  @Override
   MockYarnEngine createYarnEngine() {
     new MockYarnEngine(NODES, 8)
   }
@@ -288,7 +251,7 @@ class TestMockAppStateAAPlacement extends BaseMockAppStateTest
   public void testClusterSizeChangesDuringRequestSequence() throws Throwable {
     describe("Change the cluster size where the cluster size changes during a test sequence.")
     aaRole.desired = NODES + 1
-    List<AbstractRMOperation> operations = appState.reviewRequestAndReleaseNodes()
+    appState.reviewRequestAndReleaseNodes()
     assert aaRole.AARequestOutstanding
     assert NODES == aaRole.pendingAntiAffineRequests
     def outcome = addNewNode()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8efc4c2c/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockLabelledAAPlacement.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockLabelledAAPlacement.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockLabelledAAPlacement.groovy
index 790a80e..f0fed95 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockLabelledAAPlacement.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockLabelledAAPlacement.groovy
@@ -22,17 +22,11 @@ import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.yarn.api.records.Container
 import org.apache.hadoop.yarn.api.records.NodeState
-import org.apache.slider.providers.PlacementPolicy
-import org.apache.slider.providers.ProviderRole
-import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
-import org.apache.slider.server.appmaster.model.mock.MockFactory
 import org.apache.slider.server.appmaster.model.mock.MockNodeReport
 import org.apache.slider.server.appmaster.model.mock.MockRoles
 import org.apache.slider.server.appmaster.model.mock.MockYarnEngine
 import org.apache.slider.server.appmaster.operations.AbstractRMOperation
 import org.apache.slider.server.appmaster.state.AppState
-import org.apache.slider.server.appmaster.state.AppStateBindingInfo
-import org.apache.slider.server.appmaster.state.RoleStatus
 import org.junit.Test
 
 /**
@@ -40,45 +34,22 @@ import org.junit.Test
  */
 @CompileStatic
 @Slf4j
-class TestMockLabelledAAPlacement extends BaseMockAppStateTest
+class TestMockLabelledAAPlacement extends BaseMockAppStateAATest
     implements MockRoles {
 
-  /**
-   * Patch up a "role2" role to have anti-affinity set and the label of GPU
-   */
-  public static final ProviderRole AAROLE = new ProviderRole(
-      MockRoles.ROLE2,
-      2,
-      PlacementPolicy.ANTI_AFFINITY_REQUIRED,
-      2,
-      2,
-      "gpu")
-
-  RoleStatus aaRole
   private int NODES = 3
   private int GPU_NODES = 2
   private String HOST0 = "00000000"
   private String HOST1 = "00000001"
 
-  @Override
-  AppStateBindingInfo buildBindingInfo() {
-    def bindingInfo = super.buildBindingInfo()
-    bindingInfo.roles = [
-        MockFactory.PROVIDER_ROLE0,
-        MockFactory.PROVIDER_ROLE1,
-        AAROLE,
-    ]
-    bindingInfo
-  }
 
   @Override
   void setup() {
     super.setup()
-    aaRole = lookupRole(AAROLE.name)
     // node 1 is GPU
 
-    updateNodes(new MockNodeReport(HOST0, NodeState.RUNNING, "gpu"))
-    updateNodes(new MockNodeReport(HOST1, NodeState.RUNNING, "gpu"))
+    updateNodes(new MockNodeReport(HOST0, NodeState.RUNNING, LABEL_GPU))
+    updateNodes(new MockNodeReport(HOST1, NodeState.RUNNING, LABEL_GPU))
   }
 
   @Override
@@ -87,7 +58,7 @@ class TestMockLabelledAAPlacement extends BaseMockAppStateTest
   }
 
   void assertAllContainersAA() {
-    assertAllContainersAA(aaRole.key)
+    assertAllContainersAA(gpuRole.key)
   }
 
   /**
@@ -101,12 +72,12 @@ class TestMockLabelledAAPlacement extends BaseMockAppStateTest
              " expect the final request to be unsatisfied until the cluster changes size")
     //more than expected
     int size = GPU_NODES
-    aaRole.desired = size + 1
+    gpuRole.desired = size + 1
 
     List<AbstractRMOperation > operations = appState.reviewRequestAndReleaseNodes()
-    assert aaRole.AARequestOutstanding
+    assert gpuRole.AARequestOutstanding
 
-    assert aaRole.pendingAntiAffineRequests == size
+    assert gpuRole.pendingAntiAffineRequests == size
     for (int i = 0; i < size; i++) {
       def iter = "Iteration $i role = $aaRole"
       describe iter
@@ -127,9 +98,9 @@ class TestMockLabelledAAPlacement extends BaseMockAppStateTest
       }
     }
     // expect an outstanding AA request to be unsatisfied
-    assert aaRole.actual < aaRole.desired
-    assert !aaRole.requested
-    assert !aaRole.AARequestOutstanding
+    assert gpuRole.actual < gpuRole.desired
+    assert !gpuRole.requested
+    assert !gpuRole.AARequestOutstanding
     List<Container> allocatedContainers = engine.execute(operations, [])
     assert 0 == allocatedContainers.size()
     // in a review now, no more requests can be generated, as there is no space for AA placements,
@@ -153,10 +124,10 @@ class TestMockLabelledAAPlacement extends BaseMockAppStateTest
   @Test
   public void testClusterSizeChangesDuringRequestSequence() throws Throwable {
     describe("Change the cluster size where the cluster size changes during a test sequence.")
-    aaRole.desired = GPU_NODES + 1
+    gpuRole.desired = GPU_NODES + 1
     List<AbstractRMOperation> operations = appState.reviewRequestAndReleaseNodes()
-    assert aaRole.AARequestOutstanding
-    assert GPU_NODES == aaRole.pendingAntiAffineRequests
+    assert gpuRole.AARequestOutstanding
+    assert GPU_NODES == gpuRole.pendingAntiAffineRequests
     def outcome = addNewNode()
     assert outcome.clusterChanged
     // one call to cancel

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8efc4c2c/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockFactory.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockFactory.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockFactory.groovy
index bbd64f1..4bbfbd8 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockFactory.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockFactory.groovy
@@ -44,6 +44,9 @@ class MockFactory implements MockRoles {
   Ignore any IDE hints about needless references to the ROLE values; groovyc fails without them.
    */
 
+  /**
+   * basic role
+   */
   public static final ProviderRole PROVIDER_ROLE0 = new ProviderRole(
       MockRoles.ROLE0,
       0,
@@ -51,7 +54,9 @@ class MockFactory implements MockRoles {
       2,
       1,
       ResourceKeys.DEF_YARN_LABEL_EXPRESSION)
-  // role 1 is strict. timeout should be irrelevant; same as failures
+  /**
+   * role 1 is strict. timeout should be irrelevant; same as failures
+   */
   public static final ProviderRole PROVIDER_ROLE1 = new ProviderRole(
       MockRoles.ROLE1,
       1,
@@ -59,7 +64,10 @@ class MockFactory implements MockRoles {
       2,
       1,
       ResourceKeys.DEF_YARN_LABEL_EXPRESSION)
-  // role 2: longer delay
+
+  /**
+   * role 2: longer delay
+   */
   public static final ProviderRole PROVIDER_ROLE2 = new ProviderRole(
       MockRoles.ROLE2,
       2,
@@ -68,6 +76,28 @@ class MockFactory implements MockRoles {
       2,
       ResourceKeys.DEF_YARN_LABEL_EXPRESSION)
 
+  /**
+   * Patch up a "role2" role to have anti-affinity set
+   */
+  public static final ProviderRole AAROLE_2 = new ProviderRole(
+      MockRoles.ROLE2,
+      2,
+      PlacementPolicy.ANTI_AFFINITY_REQUIRED,
+      2,
+      2,
+      null)
+
+  /**
+   * Patch up a "role1" role to have anti-affinity set and GPI as the label
+   */
+  public static final ProviderRole AAROLE_1_GPU = new ProviderRole(
+      MockRoles.ROLE1,
+      1,
+      PlacementPolicy.ANTI_AFFINITY_REQUIRED,
+      2,
+      1,
+      MockRoles.LABEL_GPU)
+
   int appIdCount;
   int attemptIdCount;
   int containerIdCount;
@@ -83,7 +113,7 @@ class MockFactory implements MockRoles {
       PROVIDER_ROLE1,
       PROVIDER_ROLE2,
   ]
-  
+
   public static final int ROLE_COUNT = ROLES.size();
 
   MockContainerId newContainerId() {
@@ -211,6 +241,5 @@ class MockFactory implements MockRoles {
 
   MockContainerStatus newContainerStatus() {
     return new MockContainerStatus()
-    
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8efc4c2c/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockRoles.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockRoles.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockRoles.groovy
index b44482d..5ee8f24 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockRoles.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockRoles.groovy
@@ -24,4 +24,6 @@ public interface MockRoles {
   String ROLE1 = "role1"
   String ROLE2 = "role2"
   int ROLE_COUNT = 3
+  String LABEL_GPU = "gpu"
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8efc4c2c/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
index c2ea837..a4db705 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
@@ -24,8 +24,10 @@ import org.apache.hadoop.yarn.api.records.Container
 import org.apache.hadoop.yarn.api.records.Priority
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet
 import org.apache.slider.providers.ProviderService
+import org.apache.slider.server.appmaster.model.appstate.BaseMockAppStateAATest
 import org.apache.slider.server.appmaster.model.mock.*
 import org.apache.slider.server.appmaster.state.ContainerOutcome
+import org.apache.slider.server.appmaster.state.OutstandingRequest
 import org.apache.slider.server.appmaster.state.ProviderAppState
 import org.apache.slider.server.appmaster.web.WebAppApi
 import org.apache.slider.server.appmaster.web.WebAppApiImpl
@@ -34,7 +36,7 @@ import org.junit.Test
 
 @Slf4j
 //@CompileStatic
-public class TestIndexBlock extends BaseMockAppStateTest {
+public class TestIndexBlock extends BaseMockAppStateAATest {
 
   private IndexBlock indexBlock;
 
@@ -81,23 +83,51 @@ public class TestIndexBlock extends BaseMockAppStateTest {
   public void testIndex() {
     def role0 = role0Status
     def role1 = role1Status
-    role0.desired = 8
-    role0.incActual()
-    role0.incActual()
-    role0.incActual()
-    role0.incActual()
-    role0.incActual()
-    role1.incRequested()
-    role1.incRequested()
-    role1.incRequested()
+    def role2 = role2Status
+
+    def role0_desired = 8
+
+    role0.desired = role0_desired
+    int role0_actual = 5
+    int role0_requested = role0_desired - role0_actual
+    role0_actual.times {
+      role0.incActual()
+    }
+    assert role0.getActual() == role0_actual
+    role0_requested.times {
+      role0.incRequested()
+    }
+    assert role0.getRequested() == role0_requested
+
+    def role0_failures = 2
+
     role0.noteFailed(false, "", ContainerOutcome.Failed)
     role0.noteFailed(true,  "", ContainerOutcome.Failed)
 
+    // all aa roles fields are in the
+    def aarole_desired = 200
+    aaRole.desired = aarole_desired
+    def aarole_actual = 90
+    def aarole_active = 1
+    def aarole_requested = aarole_desired - aarole_actual
+    def aarole_pending = aarole_requested - 1
+    def aarole_failures = 0
+    aarole_actual.times {
+      aaRole.incActual()
+    }
+    assert aaRole.actual == aarole_actual
+    aaRole.outstandingAArequest = new OutstandingRequest(2, "")
+    // add a requested
+    aaRole.incRequested()
+    aaRole.pendingAntiAffineRequests = aarole_pending
+    assert aaRole.pendingAntiAffineRequests == aarole_pending
+
+    assert aaRole.actualAndRequested == aarole_desired
     StringWriter sw = new StringWriter(64);
     PrintWriter pw = new PrintWriter(sw);
 
     Hamlet hamlet = new Hamlet(pw, 0, false);
-    
+
     int level = hamlet.nestLevel();
     indexBlock.doIndex(hamlet, "accumulo");
 
@@ -106,17 +136,29 @@ public class TestIndexBlock extends BaseMockAppStateTest {
     assertEquals(body, level, hamlet.nestLevel())
     // verify role data came out
     assert body.contains("role0")
-    // 
-    assert body.contains("8")
-    assert body.contains("5")
-    assert body.contains("3")
-    assert body.contains("2")
-    assert body.contains("1")
-    
+    assertContains(role0_desired, body)
+    assertContains(role0_actual, body)
+    assertContains(role0_requested, body)
+    assertContains(role0_failures, body)
+
     assert body.contains("role1")
     assert body.contains("role2")
+
+    assertContains(aarole_desired, body)
+    assertContains(aarole_actual, body)
+    assertContains(aarole_requested, body)
+    assertContains(aarole_failures, body)
+
     // verify that the sorting took place
     assert body.indexOf("role0") < body.indexOf("role1")
     assert body.indexOf("role1") < body.indexOf("role2")
+
+    assert !body.contains(IndexBlock.ALL_CONTAINERS_ALLOCATED)
+    // role
+  }
+
+  def assertContains(int ex, String html) {
+    assertStringContains(Integer.toString(ex), html)
+
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8efc4c2c/slider-core/src/test/groovy/org/apache/slider/server/management/TestGauges.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/management/TestGauges.groovy b/slider-core/src/test/groovy/org/apache/slider/server/management/TestGauges.groovy
new file mode 100644
index 0000000..451bdae
--- /dev/null
+++ b/slider-core/src/test/groovy/org/apache/slider/server/management/TestGauges.groovy
@@ -0,0 +1,52 @@
+/*
+ * 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.management
+
+import org.apache.slider.server.appmaster.management.LongGauge
+import org.apache.slider.test.SliderTestBase
+import org.junit.Test
+
+class TestGauges extends  SliderTestBase {
+
+  @Test
+  public void testLongGaugeOperations() throws Throwable {
+    LongGauge gauge = new LongGauge();
+    assert gauge.get() == 0
+    gauge.inc()
+    assert gauge.get() == 1
+    gauge.inc()
+    assert gauge.get() == 2
+    gauge.inc()
+    assert gauge.get() == 3
+    assert gauge.getValue() == gauge.get()
+    assert gauge.count == gauge.get()
+
+    gauge.dec()
+    assert gauge.get() == 2
+    assert gauge.decToFloor(1) == 1
+    assert gauge.get() == 1
+    assert gauge.decToFloor(1) == 0
+    assert gauge.decToFloor(1) == 0
+    assert gauge.decToFloor(0) == 0
+
+    gauge.set(4)
+    assert gauge.decToFloor(8) == 0
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8efc4c2c/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 e1f2f75..ab81c46 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
@@ -118,11 +118,20 @@ class SliderTestUtils extends Assert {
     JsonOutput.prettyPrint(JsonOutput.toJson(src))
   }
 
+  /**
+   * Skip the test with a message
+   * @param message message logged and thrown
+   */
   public static void skip(String message) {
     log.warn("Skipping test: {}", message)
     Assume.assumeTrue(message, false);
   }
 
+  /**
+   * Skip the test with a message if condition holds
+   * @param condition predicate
+   * @param message message logged and thrown
+   */
   public static void assume(boolean condition, String message) {
     if (!condition) {
       skip(message)
@@ -132,18 +141,25 @@ class SliderTestUtils extends Assert {
   /**
    * Skip a test if not running on Windows
    */
-  public assumeWindows() {
+  public static void assumeWindows() {
     assume(Shell.WINDOWS, "not windows")
   }
 
   /**
    * Skip a test if running on Windows
    */
-  public assumeNotWindows() {
+  public static void assumeNotWindows() {
     assume(!Shell.WINDOWS, "windows")
   }
 
   /**
+   * skip a test on windows
+   */
+  public static void skipOnWindows() {
+    assumeNotWindows();
+  }
+
+  /**
    * Equality size for a list
    * @param left
    * @param right
@@ -243,15 +259,6 @@ class SliderTestUtils extends Assert {
   }
 
   /**
-   * skip a test on windows
-   */
-  public static void skipOnWindows() {
-    if (Shell.WINDOWS) {
-      skip("Not supported on windows")
-    }
-  }
-
-  /**
    * Assert that any needed libraries being present. On Unix none are needed;
    * on windows they must be present
    */
@@ -1474,6 +1481,13 @@ class SliderTestUtils extends Assert {
     }
   }
 
+  /**
+   * Probe for a metric gauge holding a value.
+   *
+   * Keys: "url:String", "gauge:String", "desiredValue:int"
+   * @param args argument map
+   * @return success on the desired value, retry if not; fail on IOE
+   */
   Outcome probeMetricGaugeValue(Map args) {
     String url = requiredMapValue(args, "url")
     String gauge = requiredMapValue(args, "gauge")
@@ -1490,4 +1504,13 @@ class SliderTestUtils extends Assert {
     }
   }
 
+  public static void assertStringContains(String expected, String text) {
+    assertNotNull("null text", text)
+    if (!text.contains(expected)) {
+      def message = "id not find $expected in \"$text\""
+      log.error(message)
+      fail(message)
+    }
+
+  }
 }