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/07 00:29:43 UTC

[01/22] incubator-slider git commit: SLIDER-82 trying to set up YarnClient, having config propagation issues in minicluster (RM client addr is 0.0.0.0)

Repository: incubator-slider
Updated Branches:
  refs/heads/feature/SLIDER-82-pass-3.1 [created] e0fb52916


SLIDER-82 trying to set up YarnClient, having config propagation issues in minicluster (RM client addr is 0.0.0.0)


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: 3b9b221f3b2140f76b8e0bc66631511b8001f965
Parents: 1312bc3
Author: Steve Loughran <st...@apache.org>
Authored: Wed Nov 4 21:12:53 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Nov 5 13:19:03 2015 +0000

----------------------------------------------------------------------
 .../slider/client/SliderYarnClientImpl.java     |  8 +++---
 .../slider/core/launch/ContainerLauncher.java   | 12 +++------
 .../server/appmaster/SliderAppMaster.java       | 27 ++++++++++----------
 3 files changed, 23 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3b9b221f/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java b/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
index 803ccd6..867603b 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
@@ -45,6 +45,7 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.net.BindException;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
@@ -69,10 +70,11 @@ public class SliderYarnClientImpl extends YarnClientImpl {
 
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
-    String addr = conf.get(YarnConfiguration.RM_ADDRESS);
-    if (addr.startsWith("0.0.0.0")) {
+    InetSocketAddress clientRpcAddress = SliderUtils.getRmAddress(conf);
+    if (!SliderUtils.isAddressDefined(clientRpcAddress)) {
       // address isn't known; fail fast
-      throw new BindException("Invalid " + YarnConfiguration.RM_ADDRESS + " value:" + addr
+      throw new BindException("Invalid " + YarnConfiguration.RM_ADDRESS
+          + " value:" + conf.get(YarnConfiguration.RM_ADDRESS)
           + " - see https://wiki.apache.org/hadoop/UnsetHostnameOrPort");
     }
     super.serviceInit(conf);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3b9b221f/slider-core/src/main/java/org/apache/slider/core/launch/ContainerLauncher.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/ContainerLauncher.java b/slider-core/src/main/java/org/apache/slider/core/launch/ContainerLauncher.java
index f8ea4ee..69b937d 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/ContainerLauncher.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/ContainerLauncher.java
@@ -55,17 +55,13 @@ public class ContainerLauncher extends AbstractLauncher {
   public UserGroupInformation setupUGI() {
     UserGroupInformation user =
       UserGroupInformation.createRemoteUser(container.getId().toString());
-    String cmIpPortStr =
-      container.getNodeId().getHost() + ":" + container.getNodeId().getPort();
-    final InetSocketAddress cmAddress =
-      NetUtils.createSocketAddr(cmIpPortStr);
+    String cmIpPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort();
+    final InetSocketAddress cmAddress = NetUtils.createSocketAddr(cmIpPortStr);
 
-    org.apache.hadoop.yarn.api.records.Token containerToken =
-      container.getContainerToken();
+    org.apache.hadoop.yarn.api.records.Token containerToken = container.getContainerToken();
     if (containerToken != null) {
       Token<ContainerTokenIdentifier> token =
-        ConverterUtils.convertFromYarn(containerToken,
-                                       cmAddress);
+        ConverterUtils.convertFromYarn(containerToken, cmAddress);
       user.addToken(token);
     }
     return user;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3b9b221f/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 e6a5bd5..1a127cf 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
@@ -502,15 +502,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     addService(executorService);
 
     addService(actionQueues);
-    // set up the YARN client. This may require patching in the RM client-API address if it
-    // is (somehow) unset server-side.
-    String clientRMaddr = conf.get(YarnConfiguration.RM_ADDRESS);
-    if (clientRMaddr.startsWith("0.0.0.0")) {
-      // address isn't known; fail fast
-      throw new BindException("Invalid " + YarnConfiguration.RM_ADDRESS + " value:" + addr
-          + " - see https://wiki.apache.org/hadoop/UnsetHostnameOrPort");
-    }
-    addService(yarnClient = new SliderYarnClientImpl());
 
     //init all child services
     super.serviceInit(conf);
@@ -542,8 +533,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
    * @param args argument list
    */
   @Override // RunService
-  public Configuration bindArgs(Configuration config, String... args) throws
-                                                                      Exception {
+  public Configuration bindArgs(Configuration config, String... args) throws Exception {
     // let the superclass process it
     Configuration superConf = super.bindArgs(config, args);
     // add the slider XML config
@@ -667,10 +657,21 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     sliderAMProvider = new SliderAMProviderService();
     initAndAddService(sliderAMProvider);
     
-    InetSocketAddress address = SliderUtils.getRmSchedulerAddress(serviceConf);
-    log.info("RM is at {}", address);
+    InetSocketAddress rmSchedulerAddress = SliderUtils.getRmSchedulerAddress(serviceConf);
+    log.info("RM is at {}", rmSchedulerAddress);
     yarnRPC = YarnRPC.create(serviceConf);
 
+    // set up the YARN client. This may require patching in the RM client-API address if it
+    // is (somehow) unset server-side.    String clientRMaddr = serviceConf.get(YarnConfiguration.RM_ADDRESS);
+    InetSocketAddress clientRpcAddress = SliderUtils.getRmAddress(serviceConf);
+    if (!SliderUtils.isAddressDefined(clientRpcAddress)) {
+      // client addr is being unset. We can lift it from the other RM APIs
+      serviceConf.set(YarnConfiguration.RM_ADDRESS,
+          String.format("%s:%d", rmSchedulerAddress.getHostString(), clientRpcAddress.getPort() ));
+    }
+    initAndAddService(yarnClient = new SliderYarnClientImpl());
+    yarnClient.start();
+
     /*
      * Extract the container ID. This is then
      * turned into an (incompete) container


[16/22] incubator-slider git commit: SLIDER-963 Write mock/unit tests for AA placement -initial test; enough for the code to be written against

Posted by st...@apache.org.
SLIDER-963 Write mock/unit tests for AA placement -initial test; enough for the code to be written against


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: 8f2786ca53cdac06abb851ca4fd49944e59019bf
Parents: 0fce42f
Author: Steve Loughran <st...@apache.org>
Authored: Fri Nov 6 15:05:23 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Nov 6 15:05:23 2015 +0000

----------------------------------------------------------------------
 .../slider/server/appmaster/state/AppState.java |  52 +++++-----
 .../appstate/TestMockAppStateAAPlacement.groovy | 104 +++++++++++++++----
 .../model/mock/BaseMockAppStateTest.groovy      |   3 +-
 .../appmaster/model/mock/MockNodeReport.groovy  |  40 +++++++
 .../appmaster/model/mock/MockYarnCluster.groovy |  15 ++-
 .../appmaster/model/mock/MockYarnEngine.groovy  |   7 ++
 6 files changed, 173 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8f2786ca/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 946d45f..29d5cde 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
@@ -529,8 +529,7 @@ public class AppState {
         // this is a new value
         log.info("Adding role {}", name);
         MapOperations resComponent = resources.getComponent(name);
-        ProviderRole dynamicRole =
-            createDynamicProviderRole(name, resComponent);
+        ProviderRole dynamicRole = createDynamicProviderRole(name, resComponent);
         buildRole(dynamicRole);
         roleList.add(dynamicRole);
       }
@@ -546,11 +545,11 @@ public class AppState {
         InternalKeys.DEFAULT_INTERNAL_CONTAINER_FAILURE_SHORTLIFE);
 
     failureThreshold = globalResOpts.getOptionInt(
-        ResourceKeys.CONTAINER_FAILURE_THRESHOLD,
-        ResourceKeys.DEFAULT_CONTAINER_FAILURE_THRESHOLD);
+        CONTAINER_FAILURE_THRESHOLD,
+        DEFAULT_CONTAINER_FAILURE_THRESHOLD);
     nodeFailureThreshold = globalResOpts.getOptionInt(
-        ResourceKeys.NODE_FAILURE_THRESHOLD,
-        ResourceKeys.DEFAULT_NODE_FAILURE_THRESHOLD);
+        NODE_FAILURE_THRESHOLD,
+        DEFAULT_NODE_FAILURE_THRESHOLD);
     initClusterStatus();
 
 
@@ -606,22 +605,21 @@ public class AppState {
    * @return a new provider role
    * @throws BadConfigException bad configuration
    */
-  public ProviderRole createDynamicProviderRole(String name,
-                                                MapOperations component) throws
-                                                        BadConfigException {
-    String priOpt = component.getMandatoryOption(ResourceKeys.COMPONENT_PRIORITY);
-    int priority = SliderUtils.parseAndValidate("value of " + name + " " +
-                                                ResourceKeys.COMPONENT_PRIORITY,
-        priOpt, 0, 1, -1);
-    String placementOpt = component.getOption(
-        ResourceKeys.COMPONENT_PLACEMENT_POLICY,
+  public ProviderRole createDynamicProviderRole(String name, MapOperations component)
+      throws BadConfigException {
+    String priOpt = component.getMandatoryOption(COMPONENT_PRIORITY);
+    int priority = SliderUtils.parseAndValidate(
+        "value of " + name + " " + COMPONENT_PRIORITY, priOpt, 0, 1, -1);
+
+    String placementOpt = component.getOption(COMPONENT_PLACEMENT_POLICY,
         Integer.toString(PlacementPolicy.DEFAULT));
-    int placement = SliderUtils.parseAndValidate("value of " + name + " " +
-        ResourceKeys.COMPONENT_PLACEMENT_POLICY,
-        placementOpt, 0, 0, -1);
-    int placementTimeout =
-        component.getOptionInt(ResourceKeys.PLACEMENT_ESCALATE_DELAY,
-            ResourceKeys.DEFAULT_PLACEMENT_ESCALATE_DELAY_SECONDS);
+
+    int placement = SliderUtils.parseAndValidate(
+        "value of " + name + " " + COMPONENT_PLACEMENT_POLICY, placementOpt, 0, 0, -1);
+
+    int placementTimeout = component.getOptionInt(PLACEMENT_ESCALATE_DELAY,
+            DEFAULT_PLACEMENT_ESCALATE_DELAY_SECONDS);
+
     ProviderRole newRole = new ProviderRole(name,
         priority,
         placement,
@@ -662,7 +660,7 @@ public class AppState {
         instanceDefinition.getResourceOperations();
     if (resources.getComponent(SliderKeys.COMPONENT_AM) != null) {
       resources.setComponentOpt(
-          SliderKeys.COMPONENT_AM, ResourceKeys.COMPONENT_INSTANCES, "1");
+          SliderKeys.COMPONENT_AM, COMPONENT_INSTANCES, "1");
     }
 
 
@@ -780,7 +778,7 @@ public class AppState {
   private int getDesiredInstanceCount(ConfTreeOperations resources,
       String role) throws BadConfigException {
     int desiredInstanceCount =
-      resources.getComponentOptInt(role, ResourceKeys.COMPONENT_INSTANCES, 0);
+      resources.getComponentOptInt(role, COMPONENT_INSTANCES, 0);
 
     if (desiredInstanceCount < 0) {
       log.error("Role {} has negative desired instances : {}", role,
@@ -1271,7 +1269,7 @@ public class AppState {
     String val = resources.getComponentOpt(name, option,
         Integer.toString(defVal));
     Integer intVal;
-    if (ResourceKeys.YARN_RESOURCE_MAX.equals(val)) {
+    if (YARN_RESOURCE_MAX.equals(val)) {
       intVal = maxVal;
     } else {
       intVal = Integer.decode(val);
@@ -1679,7 +1677,7 @@ public class AppState {
       String rolename = role.getName();
       List<String> instances = instanceMap.get(rolename);
       int nodeCount = instances != null ? instances.size(): 0;
-      cd.setRoleOpt(rolename, ResourceKeys.COMPONENT_INSTANCES,
+      cd.setRoleOpt(rolename, COMPONENT_INSTANCES,
                     role.getDesired());
       cd.setRoleOpt(rolename, RoleKeys.ROLE_ACTUAL_INSTANCES, nodeCount);
       cd.setRoleOpt(rolename, ROLE_REQUESTED_INSTANCES, role.getRequested());
@@ -1813,7 +1811,7 @@ public class AppState {
     ConfTreeOperations resources =
         instanceDefinition.getResourceOperations();
     return resources.getComponentOptInt(roleStatus.getName(),
-        ResourceKeys.CONTAINER_FAILURE_THRESHOLD,
+        CONTAINER_FAILURE_THRESHOLD,
         failureThreshold);
   }
 
@@ -1827,7 +1825,7 @@ public class AppState {
     ConfTreeOperations resources =
         instanceDefinition.getResourceOperations();
     return resources.getComponentOptInt(roleName,
-                                        ResourceKeys.NODE_FAILURE_THRESHOLD,
+                                        NODE_FAILURE_THRESHOLD,
                                         nodeFailureThreshold);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8f2786ca/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 6168146..810affc 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,21 +21,20 @@ 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.NodeId
 import org.apache.hadoop.yarn.client.api.AMRMClient
+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.MockRoles
 import org.apache.slider.server.appmaster.operations.AbstractRMOperation
 import org.apache.slider.server.appmaster.operations.CancelSingleRequest
-import org.apache.slider.server.appmaster.operations.ContainerReleaseOperation
 import org.apache.slider.server.appmaster.operations.ContainerRequestOperation
+import org.apache.slider.server.appmaster.state.AppStateBindingInfo
 import org.apache.slider.server.appmaster.state.ContainerAssignment
-import org.apache.slider.server.appmaster.state.RoleHistoryUtils
+import org.apache.slider.server.appmaster.state.NodeMap
 import org.apache.slider.server.appmaster.state.RoleInstance
 import org.junit.Test
 
-import static org.apache.slider.server.appmaster.state.ContainerPriority.extractRole
-
 /**
  * Test Anti-affine placement
  */
@@ -44,17 +43,79 @@ import static org.apache.slider.server.appmaster.state.ContainerPriority.extract
 class TestMockAppStateAAPlacement extends BaseMockAppStateTest
     implements MockRoles {
 
-//  @Test
-  public void testAllocateAA() throws Throwable {
+  static private final ProviderRole aaRole = MockFactory.PROVIDER_ROLE2
+  private static final int roleId = aaRole.id
+/*
+  @Override
+  AppStateBindingInfo buildBindingInfo() {
+    def bindingInfo = super.buildBindingInfo()
+    // only have the AA role, to avoid complications/confusion
+    bindingInfo.roles = [aaRole]
+    bindingInfo
+  }*/
+
+  /**
+   * Get the container request of an indexed entry. Includes some assertions for better diagnostics
+   * @param ops operation list
+   * @param index index in the list
+   * @return the request.
+   */
+  AMRMClient.ContainerRequest getRequest(List<AbstractRMOperation> ops, int index) {
+    assert index < ops.size()
+    def op = ops[index]
+    assert op instanceof ContainerRequestOperation
+    ((ContainerRequestOperation) op).request
+  }
 
-    def aaRole = role2Status
+  /**
+   * Get the cancel request of an indexed entry. Includes some assertions for better diagnostics
+   * @param ops operation list
+   * @param index index in the list
+   * @return the request.
+   */
+  AMRMClient.ContainerRequest getCancel(List<AbstractRMOperation> ops, int index) {
+    assert index < ops.size()
+    def op = ops[index]
+    assert op instanceof CancelSingleRequest
+    ((CancelSingleRequest) op).request
+  }
 
+  /**
+   * Get the single request of a list of operations; includes the check for the size
+   * @param ops operations list of size 1
+   * @return the request within the first ContainerRequestOperation
+   */
+  public AMRMClient.ContainerRequest getSingleRequest(List<AbstractRMOperation> ops) {
+    assert 1 == ops.size()
+    getRequest(ops, 0)
+  }
+  /**
+   * Get the single request of a list of operations; includes the check for the size
+   * @param ops operations list of size 1
+   * @return the request within the first operation
+   */
+  public AMRMClient.ContainerRequest getSingleCancel(List<AbstractRMOperation> ops) {
+    assert 1 == ops.size()
+    getCancel(ops, 0)
+  }
+
+  @Test
+  public void testVerifyNodeMap() throws Throwable {
+
+    def nodemap = appState.roleHistory.cloneNodemap()
+    assert nodemap.size() > 0
+  }
+
+  @Test
+  public void testAllocateAANoLabel() throws Throwable {
+
+    def aaRole = lookupRole(aaRole.name)
+
+    // want two instances, so there will be two iterations
     aaRole.desired = 2
 
     List<AbstractRMOperation> ops = appState.reviewRequestAndReleaseNodes()
-    assert 1 == ops.size()
-    ContainerRequestOperation operation = (ContainerRequestOperation) ops[0]
-    AMRMClient.ContainerRequest request = operation.request
+    AMRMClient.ContainerRequest request = getSingleRequest(ops)
     assert request.relaxLocality
     assert request.nodes == null
     assert request.racks == null
@@ -68,18 +129,22 @@ class TestMockAppStateAAPlacement extends BaseMockAppStateTest
 
     // notify the container ane expect
     List<ContainerAssignment> assignments = [];
-    List<AbstractRMOperation> releaseOperations = []
-    appState.onContainersAllocated([allocated], assignments, releaseOperations)
+    List<AbstractRMOperation> operations = []
+    appState.onContainersAllocated([allocated], assignments, operations)
 
-    // verify the release matches the allocation
-    assert releaseOperations.size() == 1
-    CancelSingleRequest cancelOp = releaseOperations[0] as CancelSingleRequest;
-    assert cancelOp.request.capability.equals(allocated.resource)
-    // now the assignment
+    // assignment
     assert assignments.size() == 1
 
+    // verify the release matches the allocation
+    assert operations.size() == 2
+    assert getCancel(operations, 0).capability.equals(allocated.resource)
+
     // we also expect a new allocation request to have been issued
-    //
+
+    def req2 = getRequest(operations, 1)
+    // now the nodes should be a list
+    Container allocated2 = engine.allocateContainer(req2)
+
 
     ContainerAssignment assigned = assignments[0]
     Container container = assigned.container
@@ -89,6 +154,7 @@ class TestMockAppStateAAPlacement extends BaseMockAppStateTest
     assert appState.onNodeManagerContainerStarted(container.id)
     ops = appState.reviewRequestAndReleaseNodes()
     assert ops.size() == 0
+
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8f2786ca/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
index 40d7fd7..44d35be 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
@@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.api.records.Container
 import org.apache.hadoop.yarn.api.records.ContainerId
 import org.apache.hadoop.yarn.api.records.ContainerState
 import org.apache.hadoop.yarn.api.records.ContainerStatus
+import org.apache.hadoop.yarn.api.records.NodeReport
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.slider.common.tools.SliderFileSystem
 import org.apache.slider.common.tools.SliderUtils
@@ -88,7 +89,6 @@ abstract class BaseMockAppStateTest extends SliderTestBase implements MockRoles
         applicationId: applicationId,
         attemptId: 1)
 
-    fs = HadoopFS.get(new URI("file:///"), conf)
     historyWorkDir = new File("target/history", historyDirName)
     historyPath = new Path(historyWorkDir.toURI())
     fs.delete(historyPath, true)
@@ -108,6 +108,7 @@ abstract class BaseMockAppStateTest extends SliderTestBase implements MockRoles
     binding.roles = factory.ROLES
     binding.fs = fs
     binding.historyPath = historyPath
+    binding.nodeReports = engine.nodeReports as List<NodeReport>
     binding
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8f2786ca/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockNodeReport.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockNodeReport.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockNodeReport.groovy
new file mode 100644
index 0000000..1c7a816
--- /dev/null
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockNodeReport.groovy
@@ -0,0 +1,40 @@
+/*
+ * 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.mock
+
+import org.apache.hadoop.yarn.api.records.NodeId
+import org.apache.hadoop.yarn.api.records.NodeReport
+import org.apache.hadoop.yarn.api.records.NodeState
+import org.apache.hadoop.yarn.api.records.Resource
+
+/**
+ * Node report for testing
+ */
+class MockNodeReport extends NodeReport {
+  NodeId nodeId;
+  NodeState nodeState;
+  String httpAddress;
+  String rackName;
+  Resource used;
+  Resource capability;
+  int numContainers;
+  String healthReport;
+  long lastHealthReportTime;
+  Set<String> nodeLabels;
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8f2786ca/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnCluster.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnCluster.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnCluster.groovy
index 99a9213..265a796 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnCluster.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnCluster.groovy
@@ -22,6 +22,7 @@ import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.yarn.api.records.ContainerId
 import org.apache.hadoop.yarn.api.records.NodeId
+import org.apache.hadoop.yarn.api.records.NodeState
 
 /**
  * Models the cluster itself: a set of mock cluster nodes.
@@ -143,7 +144,14 @@ public class MockYarnCluster {
     }
     return total;
   }
-  
+
+  /**
+   * Get the list of node reports. These are not cloned; updates will persist in the nodemap
+   * @return current node report list
+   */
+  List<MockNodeReport> getNodeReports() {
+    nodes.collect { MockYarnClusterNode n -> n.nodeReport }
+  }
   
 /**
  * Model cluster nodes on the simpler "slot" model than the YARN-era
@@ -159,6 +167,7 @@ public class MockYarnCluster {
     public final MockNodeId nodeId;
     public final MockYarnClusterContainer[] containers;
     private boolean offline;
+    public MockNodeReport nodeReport
 
     public MockYarnClusterNode(int index, int size) {
       nodeIndex = index;
@@ -171,6 +180,10 @@ public class MockYarnCluster {
         MockContainerId mci = new MockContainerId(containerId: cid)
         containers[i] = new MockYarnClusterContainer(mci)
       }
+
+      nodeReport = new MockNodeReport()
+      nodeReport.nodeId = nodeId
+      nodeReport.nodeState = NodeState.RUNNING
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8f2786ca/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnEngine.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnEngine.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnEngine.groovy
index 5860c6b..965219d 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnEngine.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnEngine.groovy
@@ -157,4 +157,11 @@ class MockYarnEngine {
     }
   }
 
+  /**
+   * Get the list of node reports. These are not cloned; updates will persist in the nodemap
+   * @return current node report list
+   */
+  List<MockNodeReport> getNodeReports() {
+    cluster.nodeReports
+  }
 }
\ No newline at end of file


[06/22] incubator-slider git commit: SLIDER-947 build node map from yarn update reports; serve via REST/IPC —this is done with YarnClient and building the initial map at launch time. Tests do not yet do this.

Posted by st...@apache.org.
SLIDER-947 build node map from yarn update reports; serve via REST/IPC —this is done with YarnClient and building the initial map at launch time. Tests do not yet do this.


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: 5887dde8e68b5d650d692d6e1211a7382465c962
Parents: f1bad85
Author: Steve Loughran <st...@apache.org>
Authored: Thu Nov 5 14:15:54 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Nov 5 14:15:54 2015 +0000

----------------------------------------------------------------------
 .../org/apache/slider/client/SliderClient.java  | 21 +++++++++++---------
 .../core/launch/JavaCommandLineBuilder.java     | 17 ++++++++++++++--
 .../server/appmaster/SliderAppMaster.java       |  1 +
 .../slider/server/appmaster/state/AppState.java |  5 ++++-
 .../appmaster/state/AppStateBindingInfo.java    |  1 +
 .../server/appmaster/state/RoleHistory.java     |  7 ++++---
 .../slider/agent/rest/TestStandaloneREST.groovy |  7 +++----
 7 files changed, 40 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5887dde8/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 07c915c..ea6810b 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
@@ -2103,9 +2103,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       commandLine.add(Arguments.ARG_FILESYSTEM, serviceArgs.getFilesystemBinding());
     }
 
-    /**
-     * pass the registry binding
-     */
+    // pass the registry binding
     commandLine.addConfOptionToCLI(config, RegistryConstants.KEY_REGISTRY_ZK_ROOT,
         RegistryConstants.DEFAULT_ZK_REGISTRY_ROOT);
     commandLine.addMandatoryConfOption(config, RegistryConstants.KEY_REGISTRY_ZK_QUORUM);
@@ -2115,6 +2113,15 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       // the relevant security settings go over
       commandLine.addConfOption(config, DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY);
     }
+
+    // copy over any/all YARN RM client values, in case the server-side XML conf file
+    // has the 0.0.0.0 address
+    commandLine.addConfOptions(config,
+        YarnConfiguration.RM_ADDRESS,
+        YarnConfiguration.RM_CLUSTER_ID,
+        YarnConfiguration.RM_HOSTNAME,
+        YarnConfiguration.RM_PRINCIPAL);
+
     // write out the path output
     commandLine.addOutAndErrFiles(STDOUT_AM, STDERR_AM);
 
@@ -2129,12 +2136,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
 
 
     // Set the priority for the application master
-
-    int amPriority = config.getInt(KEY_YARN_QUEUE_PRIORITY,
-                                   DEFAULT_YARN_QUEUE_PRIORITY);
-
-
-    amLauncher.setPriority(amPriority);
+    amLauncher.setPriority(config.getInt(KEY_YARN_QUEUE_PRIORITY,
+                                   DEFAULT_YARN_QUEUE_PRIORITY));
 
     // Set the queue to which this application is to be submitted in the RM
     // Queue for App master

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5887dde8/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java b/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
index 9197e5d..ccb610a 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
@@ -25,6 +25,8 @@ import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.core.exceptions.BadConfigException;
 
+import java.util.Map;
+
 /**
  * Command line builder purely for the Java CLI.
  * Some of the <code>define</code> methods are designed to work with Hadoop tool and
@@ -86,8 +88,18 @@ public class JavaCommandLineBuilder extends CommandLineBuilder {
   }
 
   public boolean addConfOption(Configuration conf, String key) {
-    String val = conf.get(key);
-    return defineIfSet(key, val);
+    return defineIfSet(key, conf.get(key));
+  }
+
+  /**
+   * Add a varargs list of configuration parameters —if they are present
+   * @param conf configuration source
+   * @param keys keys
+   */
+  public void addConfOptions(Configuration conf, String...keys) {
+    for (String key : keys) {
+      addConfOption(conf, key);
+    }
   }
 
   public String addConfOptionToCLI(Configuration conf,
@@ -137,4 +149,5 @@ public class JavaCommandLineBuilder extends CommandLineBuilder {
       throw new BadConfigException("Missing configuration option: " + key);
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5887dde8/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 1a127cf..d74688b 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
@@ -666,6 +666,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     InetSocketAddress clientRpcAddress = SliderUtils.getRmAddress(serviceConf);
     if (!SliderUtils.isAddressDefined(clientRpcAddress)) {
       // client addr is being unset. We can lift it from the other RM APIs
+      log.warn("Yarn RM address was unbound; attempting to fix up");
       serviceConf.set(YarnConfiguration.RM_ADDRESS,
           String.format("%s:%d", rmSchedulerAddress.getHostString(), clientRpcAddress.getPort() ));
     }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5887dde8/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 1325148..9e29af2 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
@@ -541,7 +541,7 @@ public class AppState {
 
     //set the livespan
     MapOperations globalResOpts = instanceDefinition.getResourceOperations().getGlobalOptions();
-    
+
     startTimeThreshold = globalResOpts.getOptionInt(
         InternalKeys.INTERNAL_CONTAINER_FAILURE_SHORTLIFE,
         InternalKeys.DEFAULT_INTERNAL_CONTAINER_FAILURE_SHORTLIFE);
@@ -559,6 +559,9 @@ public class AppState {
     roleHistory = new RoleHistory(roleList);
     roleHistory.register(metricsAndMonitoring);
     roleHistory.onStart(binding.fs, binding.historyPath);
+    // trigger first node update
+    roleHistory.onNodesUpdated(binding.nodeReports);
+
 
     //rebuild any live containers
     rebuildModelFromRestart(binding.liveContainers);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5887dde8/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java
index a2a0b60..a4a9b7e 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java
@@ -58,5 +58,6 @@ public class AppStateBindingInfo {
     Preconditions.checkArgument(roles != null, "null providerRoles");
     Preconditions.checkArgument(fs != null, "null fs");
     Preconditions.checkArgument(historyPath != null, "null historyDir");
+    Preconditions.checkArgument(nodeReports != null, "null nodeReports");
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5887dde8/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
index d9a6b34..53c2689 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
@@ -76,7 +76,8 @@ public class RoleHistory {
   /** Time when saved */
   private final Timestamp saveTime = new Timestamp(0);
 
-  /** If the history was loaded, the time at which the history was saved */
+  /** If the history was loaded, the time at which the history was saved.
+   * That is: the time the data was valid */
   private final Timestamp thawedDataTime = new Timestamp(0);
   
   private NodeMap nodemap;
@@ -817,12 +818,12 @@ public class RoleHistory {
       if (hostname == null || nodeState == null) {
         continue;
       }
+      log.debug("host {} is in state {}", hostname, nodeState);
       // update the node; this also creates an instance if needed
       boolean updated = nodemap.updateNode(hostname, updatedNode);
       if (updated) {
-        log.debug("Updated host {} to state {}", hostname, nodeState);
         if (nodeState.isUnusable()) {
-          log.info("Failed node {}", hostname);
+          log.info("Failed node {} state {}", hostname, nodeState);
           failedNodes.add(hostname);
         } else {
           failedNodes.remove(hostname);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5887dde8/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
index 29fa51a..7cb1837 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
@@ -71,7 +71,7 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
     execOperation(WEB_STARTUP_TIME) {
       GET(directAM)
     }
-    
+
     execOperation(WEB_STARTUP_TIME) {
       def metrics = GET(directAM, SYSTEM_METRICS_JSON)
       log.info prettyPrintJson(metrics)
@@ -84,14 +84,13 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
     log.info GET(proxyAM, SYSTEM_HEALTHCHECK)
     log.info GET(proxyAM, SYSTEM_METRICS_JSON)
 
-    // using the metrics, await the first node status update
-    /* SLIDER--82: disabled
+    // using the metrics, await the first node status update.
+    // this should be from AM launch itself
     awaitGaugeValue(
         appendToURL(proxyAM, SYSTEM_METRICS_JSON),
         "org.apache.slider.server.appmaster.state.RoleHistory.nodes-updated.flag",
         1,
         WEB_STARTUP_TIME  * 2, 500)
-     */
 
     // Is the back door required? If so, don't test complex verbs via the proxy
     def proxyComplexVerbs = !SliderXmlConfKeys.X_DEV_INSECURE_REQUIRED


[22/22] incubator-slider git commit: SLIDER-966 when flexing down, pending requests are decremented before trying to cancel anything outstanding

Posted by st...@apache.org.
SLIDER-966 when flexing down, pending requests are decremented before trying to cancel anything outstanding


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: e0fb529161ae2fa70dc719af4ccc35ee0e5f9c1f
Parents: ee0c8da
Author: Steve Loughran <st...@apache.org>
Authored: Fri Nov 6 23:27:04 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Nov 6 23:27:04 2015 +0000

----------------------------------------------------------------------
 .../operations/AbstractRMOperation.java         |  2 +-
 .../slider/server/appmaster/state/AppState.java | 45 ++++++++++++++++--
 .../appstate/TestMockAppStateAAPlacement.groovy | 50 +++++++++++++++++++-
 3 files changed, 91 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e0fb5291/slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AbstractRMOperation.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AbstractRMOperation.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AbstractRMOperation.java
index b5b27c5..ed3f197 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AbstractRMOperation.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AbstractRMOperation.java
@@ -26,5 +26,5 @@ public abstract class AbstractRMOperation {
    * @param handler handler to perform the execution
    */
   public abstract void execute(RMOperationHandlerActions handler);
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e0fb5291/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 53ab2fe..21f59a1 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
@@ -84,9 +84,33 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import static org.apache.slider.api.ResourceKeys.*;
-import static org.apache.slider.api.RoleKeys.*;
-import static org.apache.slider.api.StateValues.*;
+import static org.apache.slider.api.ResourceKeys.COMPONENT_INSTANCES;
+import static org.apache.slider.api.ResourceKeys.COMPONENT_PLACEMENT_POLICY;
+import static org.apache.slider.api.ResourceKeys.COMPONENT_PRIORITY;
+import static org.apache.slider.api.ResourceKeys.CONTAINER_FAILURE_THRESHOLD;
+import static org.apache.slider.api.ResourceKeys.DEFAULT_CONTAINER_FAILURE_THRESHOLD;
+import static org.apache.slider.api.ResourceKeys.DEFAULT_NODE_FAILURE_THRESHOLD;
+import static org.apache.slider.api.ResourceKeys.DEFAULT_PLACEMENT_ESCALATE_DELAY_SECONDS;
+import static org.apache.slider.api.ResourceKeys.DEF_YARN_CORES;
+import static org.apache.slider.api.ResourceKeys.DEF_YARN_LABEL_EXPRESSION;
+import static org.apache.slider.api.ResourceKeys.DEF_YARN_MEMORY;
+import static org.apache.slider.api.ResourceKeys.NODE_FAILURE_THRESHOLD;
+import static org.apache.slider.api.ResourceKeys.PLACEMENT_ESCALATE_DELAY;
+import static org.apache.slider.api.ResourceKeys.YARN_CORES;
+import static org.apache.slider.api.ResourceKeys.YARN_LABEL_EXPRESSION;
+import static org.apache.slider.api.ResourceKeys.YARN_MEMORY;
+import static org.apache.slider.api.ResourceKeys.YARN_RESOURCE_MAX;
+import static org.apache.slider.api.RoleKeys.ROLE_FAILED_INSTANCES;
+import static org.apache.slider.api.RoleKeys.ROLE_FAILED_RECENTLY_INSTANCES;
+import static org.apache.slider.api.RoleKeys.ROLE_FAILED_STARTING_INSTANCES;
+import static org.apache.slider.api.RoleKeys.ROLE_NODE_FAILED_INSTANCES;
+import static org.apache.slider.api.RoleKeys.ROLE_PREEMPTED_INSTANCES;
+import static org.apache.slider.api.RoleKeys.ROLE_RELEASING_INSTANCES;
+import static org.apache.slider.api.RoleKeys.ROLE_REQUESTED_INSTANCES;
+import static org.apache.slider.api.StateValues.STATE_CREATED;
+import static org.apache.slider.api.StateValues.STATE_DESTROYED;
+import static org.apache.slider.api.StateValues.STATE_LIVE;
+import static org.apache.slider.api.StateValues.STATE_SUBMITTED;
 
 
 /**
@@ -1909,6 +1933,21 @@ public class AppState {
       // reduce the number expected (i.e. subtract the delta)
       long excess = -delta;
 
+      if (isAA) {
+        // there may be pending requests which can be cancelled here
+        long pending = role.getPendingAntiAffineRequests();
+        if (excess <= pending) {
+          long outstanding = pending - excess;
+          log.info("Cancelling {} pending AA allocations, leaving {}", excess, outstanding);
+          role.setPendingAntiAffineRequests(outstanding);
+          excess = 0;
+        } else {
+          // not enough
+          log.info("Cancelling all pending AA allocations");
+          role.setPendingAntiAffineRequests(0);
+          excess -= pending;
+        }
+      }
       // how many requests are outstanding?
       long outstandingRequests = role.getRequested();
       if (outstandingRequests > 0) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e0fb5291/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 928e355..baf88dc 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
@@ -154,9 +154,7 @@ class TestMockAppStateAAPlacement extends BaseMockAppStateTest
     assert 2 == ops2.size()
     assert aaRole.pendingAntiAffineRequests == 1
 
-
     assert 0 == appState.reviewRequestAndReleaseNodes().size()
-
     // now trigger the next execution cycle
     List<AbstractRMOperation> ops3 = []
     assert 1  == submitOperations(ops2, [], ops3).size()
@@ -164,4 +162,52 @@ class TestMockAppStateAAPlacement extends BaseMockAppStateTest
     assert aaRole.pendingAntiAffineRequests == 0
   }
 
+  @Test
+  public void testAllocateFlexDown() throws Throwable {
+    // want multiple instances, so there will be iterations
+    aaRole.desired = 2
+    List<AbstractRMOperation> ops = appState.reviewRequestAndReleaseNodes()
+    getSingleRequest(ops)
+    assert aaRole.pendingAntiAffineRequests == 1
+
+    // flex down so that the next request should be cancelled
+    aaRole.desired = 1
+
+    // expect: no new reqests, pending count --
+    List<AbstractRMOperation> ops2 = appState.reviewRequestAndReleaseNodes()
+    assert ops2.empty
+    assert aaRole.pendingAntiAffineRequests == 0
+
+    // next iter
+    submitOperations(ops, [], ops2).size()
+    assert 1 == ops2.size()
+  }
+
+  /**
+   * Here flex down while there is only one outstanding request.
+   * The outstanding flex should be cancelled
+   * @throws Throwable
+   */
+  @Test
+  public void testAllocateFlexDownForcesCancel() throws Throwable {
+    // want multiple instances, so there will be iterations
+    aaRole.desired = 1
+    List<AbstractRMOperation> ops = appState.reviewRequestAndReleaseNodes()
+    getSingleRequest(ops)
+    assert aaRole.pendingAntiAffineRequests == 0
+
+    // flex down so that the next request should be cancelled
+    aaRole.desired = 0
+    // expect: no new reqests, pending count --
+    List<AbstractRMOperation> ops2 = appState.reviewRequestAndReleaseNodes()
+    assert aaRole.pendingAntiAffineRequests == 0
+    assert ops2.size() == 1
+    getSingleCancel(ops2)
+
+    // next iter
+    submitOperations(ops, [], ops2).size()
+    assert 1 == ops2.size()
+  }
+
+
 }


[18/22] incubator-slider git commit: SLIDER-965 RoleStatus and AppState move to using LongGauges to store numbers in

Posted by st...@apache.org.
SLIDER-965 RoleStatus and AppState move to using LongGauges to store numbers in


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: ac98d826e0e0bc21590f8b25aa4366707b054de2
Parents: 5b7f6dd
Author: Steve Loughran <st...@apache.org>
Authored: Fri Nov 6 21:17:25 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Nov 6 21:17:25 2015 +0000

----------------------------------------------------------------------
 .../slider/api/proto/RestTypeMarshalling.java   |  1 -
 .../slider/api/types/ComponentInformation.java  |  1 -
 .../server/appmaster/management/LongGauge.java  | 42 ++++++++++---
 .../slider/server/appmaster/state/AppState.java | 42 +++++--------
 .../server/appmaster/state/RoleStatus.java      | 62 ++++++++++++++++----
 5 files changed, 101 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ac98d826/slider-core/src/main/java/org/apache/slider/api/proto/RestTypeMarshalling.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/api/proto/RestTypeMarshalling.java b/slider-core/src/main/java/org/apache/slider/api/proto/RestTypeMarshalling.java
index 115405c..b7985e6 100644
--- a/slider-core/src/main/java/org/apache/slider/api/proto/RestTypeMarshalling.java
+++ b/slider-core/src/main/java/org/apache/slider/api/proto/RestTypeMarshalling.java
@@ -85,7 +85,6 @@ public class RestTypeMarshalling {
       info.failureMessage = wire.getFailureMessage();
     }
     info.pendingAntiAffineRequestCount = wire.getPendingAntiAffineRequestCount();
-    info.pendingAntiAffineRequest = info.pendingAntiAffineRequestCount > 0;
     return info;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ac98d826/slider-core/src/main/java/org/apache/slider/api/types/ComponentInformation.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/api/types/ComponentInformation.java b/slider-core/src/main/java/org/apache/slider/api/types/ComponentInformation.java
index 3b4b8bd..9d8a4ee 100644
--- a/slider-core/src/main/java/org/apache/slider/api/types/ComponentInformation.java
+++ b/slider-core/src/main/java/org/apache/slider/api/types/ComponentInformation.java
@@ -52,7 +52,6 @@ public class ComponentInformation {
   public int requested;
   public int failed, started, startFailed, completed, totalRequested;
   public int nodeFailed, failedRecently, preempted;
-  public boolean pendingAntiAffineRequest;
   public int pendingAntiAffineRequestCount;
 
   public String failureMessage;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ac98d826/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 72a8805..ac9ac0e 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
@@ -25,7 +25,9 @@ import java.util.concurrent.atomic.AtomicLong;
 
 /**
  * This is a {@link AtomicLong} which acts as a metrics gauge: its state can be exposed as
- * a management value.
+ * a metrics.
+ * It also exposes some of the same method names as the Codahale Counter class, so that
+ * it's easy to swap in.
  *
  */
 public class LongGauge extends AtomicLong implements Metric, Gauge<Long> {
@@ -45,22 +47,46 @@ public class LongGauge extends AtomicLong implements Metric, Gauge<Long> {
     this(0);
   }
 
-
+  /**
+   * Get the value as a metric
+   * @return current value
+   */
   @Override
   public Long getValue() {
     return get();
   }
 
+  public Long getCount() {
+    return get();
+  }
+
+  /**
+   * {@code ++}
+   */
+  public void inc() {
+    incrementAndGet();
+  }
+  /**
+   * {@code --}
+   */
+  public void dec() {
+    decrementAndGet();
+  }
+
   /**
-   * Decrement to the floor of 0.
-   * There's checks to stop more than one thread being in this method at the time, but
-   * that doesn't stop other operations on the value
+   * Decrement to the floor of 0. Operations in parallel may cause confusion here,
+   * but it will still never go below zero
    * @param delta delta
    * @return the current value
    */
-  public synchronized long decToFloor(long delta) {
-    long newval = Math.max(0L, get() - delta);
-    set(newval);
+  public long decToFloor(long delta) {
+    long l = get();
+    long r = l - delta;
+    if (r < 0) {
+      r = 0;
+    }
+    // if this fails, the decrement has been lost
+    compareAndSet(l, r);
     return get();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ac98d826/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 c46177a..f6fe474 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
@@ -19,6 +19,7 @@
 package org.apache.slider.server.appmaster.state;
 
 import com.codahale.metrics.Counter;
+import com.codahale.metrics.Metric;
 import com.codahale.metrics.MetricRegistry;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
@@ -62,6 +63,7 @@ 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.LongGauge;
 import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
 import org.apache.slider.server.appmaster.management.MetricsConstants;
 import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
@@ -194,33 +196,33 @@ public class AppState {
   /**
    * Counter for completed containers ( complete denotes successful or failed )
    */
-  private final Counter completedContainerCount = new Counter();
+  private final LongGauge completedContainerCount = new LongGauge();
 
   /**
    *   Count of failed containers
    */
-  private final Counter failedContainerCount = new Counter();
+  private final LongGauge failedContainerCount = new LongGauge();
 
   /**
    * # of started containers
    */
-  private final Counter startedContainers = new Counter();
+  private final LongGauge startedContainers = new LongGauge();
 
   /**
    * # of containers that failed to start 
    */
-  private final Counter startFailedContainerCount = new Counter();
+  private final LongGauge startFailedContainerCount = new LongGauge();
 
   /**
    * Track the number of surplus containers received and discarded
    */
-  private final Counter surplusContainers = new Counter();
+  private final LongGauge surplusContainers = new LongGauge();
 
 
   /**
    * Track the number of requested Containers
    */
-  private final Counter outstandingContainerRequests = new Counter();
+  private final LongGauge outstandingContainerRequests = new LongGauge();
 
   /**
    * Map of requested nodes. This records the command used to start it,
@@ -1211,22 +1213,10 @@ public class AppState {
   }
 
   /**
-   * dec requested count of a role
-   * <p>
-   *   Also updates application state counters.
-   * @param role role to decrement
-   */
-  protected synchronized void decrementRequestCount(RoleStatus role) {
-    role.decRequested();
-  }
-
-  /**
    * Inc #of outstanding requests.
    */
   private void incOutstandingContainerRequests() {
-    synchronized (outstandingContainerRequests) {
-      outstandingContainerRequests.inc();
-    }
+     outstandingContainerRequests.inc();
   }
 
   /**
@@ -1700,7 +1690,7 @@ public class AppState {
    */  
   public ApplicationLivenessInformation getApplicationLivenessInformation() {
     ApplicationLivenessInformation li = new ApplicationLivenessInformation();
-    int outstanding = (int) outstandingContainerRequests.getCount();
+    int outstanding = outstandingContainerRequests.intValue();
     li.requestsOutstanding = outstanding;
     li.allRequestsSatisfied = outstanding <= 0;
     return li;
@@ -1716,15 +1706,15 @@ public class AppState {
     sliderstats.put(StatusKeys.STATISTICS_CONTAINERS_LIVE,
         liveNodes.size());
     sliderstats.put(StatusKeys.STATISTICS_CONTAINERS_COMPLETED,
-        (int)completedContainerCount.getCount());
+        completedContainerCount.intValue());
     sliderstats.put(StatusKeys.STATISTICS_CONTAINERS_FAILED,
-        (int)failedContainerCount.getCount());
+        failedContainerCount.intValue());
     sliderstats.put(StatusKeys.STATISTICS_CONTAINERS_STARTED,
-        (int)startedContainers.getCount());
+        startedContainers.intValue());
     sliderstats.put(StatusKeys.STATISTICS_CONTAINERS_START_FAILED,
-        (int) startFailedContainerCount.getCount());
+         startFailedContainerCount.intValue());
     sliderstats.put(StatusKeys.STATISTICS_CONTAINERS_SURPLUS,
-        (int)surplusContainers.getCount());
+        surplusContainers.intValue());
     sliderstats.put(StatusKeys.STATISTICS_CONTAINERS_UNKNOWN_COMPLETED,
         completionOfUnknownContainerEvent.get());
     return sliderstats;
@@ -2085,7 +2075,7 @@ public class AppState {
       final RoleStatus role = lookupRoleStatus(container);
 
       //dec requested count
-      decrementRequestCount(role);
+      role.decRequested();
 
       //inc allocated count -this may need to be dropped in a moment,
       // but us needed to update the logic below

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ac98d826/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
index 4197c4f..cba963c 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
@@ -18,6 +18,8 @@
 
 package org.apache.slider.server.appmaster.state;
 
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.slider.api.types.ComponentInformation;
 import org.apache.slider.providers.PlacementPolicy;
 import org.apache.slider.providers.ProviderRole;
@@ -60,17 +62,17 @@ public final class RoleStatus implements Cloneable {
   private final LongGauge failedRecently = new LongGauge(0);
   private final LongGauge limitsExceeded = new LongGauge(0);
 
-  /** flag set to true if there is an outstanding anti-affine request */
-  private final AtomicBoolean pendingAARequest = new AtomicBoolean(false);
+  /** resource requirements */
+  private Resource resourceRequirements;
 
   /**
    * Number of AA requests queued. These should be reduced first on a
    * flex down.
    */
-  private int pendingAntiAffineRequestCount = 0;
+  private final LongGauge pendingAntiAffineRequests = new LongGauge(0);
 
   /** any pending AA request */
-  public OutstandingRequest outstandingAArequest = null;
+  private OutstandingRequest outstandingAArequest = null;
 
   private String failureMessage = "";
 
@@ -163,9 +165,8 @@ public final class RoleStatus implements Cloneable {
     return requested.incrementAndGet();
   }
 
-  
-  public long cancel(long count) {
-    return requested.decToFloor(count);
+  public void cancel(long count) {
+    requested.decToFloor(count);
   }
   
   public void decRequested() {
@@ -204,6 +205,10 @@ public final class RoleStatus implements Cloneable {
     return limitsExceeded.get();
   }
 
+  public long incPendingAntiAffineRequests(long v) {
+    return pendingAntiAffineRequests.addAndGet(v);
+  }
+
   /**
    * Note that a role failed, text will
    * be used in any diagnostics if an exception
@@ -285,6 +290,22 @@ public final class RoleStatus implements Cloneable {
     return nodeFailed.get();
   }
 
+  public long getPendingAntiAffineRequests() {
+    return pendingAntiAffineRequests.get();
+  }
+
+  public void setPendingAntiAffineRequests(long pendingAntiAffineRequests) {
+    this.pendingAntiAffineRequests.set(pendingAntiAffineRequests);
+  }
+
+  public OutstandingRequest getOutstandingAArequest() {
+    return outstandingAArequest;
+  }
+
+  public void setOutstandingAArequest(OutstandingRequest outstandingAArequest) {
+    this.outstandingAArequest = outstandingAArequest;
+  }
+
   /**
    * Get the number of roles we are short of.
    * nodes released are ignored.
@@ -321,7 +342,7 @@ public final class RoleStatus implements Cloneable {
            ", actual=" + actual +
            ", requested=" + requested +
            ", releasing=" + releasing +
-           ", pendingAntiAffineRequestCount=" + pendingAntiAffineRequestCount +
+           ", pendingAntiAffineRequestCount=" + pendingAntiAffineRequests +
            ", failed=" + failed +
            ", failed recently=" + failedRecently.get() +
            ", node failed=" + nodeFailed.get() +
@@ -376,8 +397,7 @@ public final class RoleStatus implements Cloneable {
     info.failedRecently = failedRecently.intValue();
     info.nodeFailed = nodeFailed.intValue();
     info.preempted = preempted.intValue();
-    info.pendingAntiAffineRequest = pendingAARequest.get();
-    info.pendingAntiAffineRequestCount = pendingAntiAffineRequestCount;
+    info.pendingAntiAffineRequestCount = pendingAntiAffineRequests.intValue();
     return info;
   }
 
@@ -389,6 +409,14 @@ public final class RoleStatus implements Cloneable {
     return providerRole.labelExpression;
   }
 
+  public Resource getResourceRequirements() {
+    return resourceRequirements;
+  }
+
+  public void setResourceRequirements(Resource resourceRequirements) {
+    this.resourceRequirements = resourceRequirements;
+  }
+
   /**
    * Compare two role status entries by name
    */
@@ -410,5 +438,17 @@ public final class RoleStatus implements Cloneable {
       return (o1.getKey() < o2.getKey() ? -1 : (o1.getKey() == o2.getKey() ? 0 : 1));
     }
   }
-  
+
+  /**
+   * Given a resource, set its requirements to those this role needs
+   * @param resource resource to configure
+   * @return the resource
+   */
+  public Resource copyResourceRequirements(Resource resource) {
+    Preconditions.checkNotNull(resourceRequirements,
+        "Role resource requirements have not been set");
+    resource.setMemory(resourceRequirements.getMemory());
+    resource.setVirtualCores(resourceRequirements.getVirtualCores());
+    return resource;
+  }
 }


[13/22] incubator-slider git commit: SLIDER-82: label information is set in the ProviderRole, no longer directly managed in AppState. Needed for building a sequence of AA placement requests

Posted by st...@apache.org.
SLIDER-82: label information is set in the ProviderRole, no longer directly managed in AppState. Needed for building a sequence of AA placement requests


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: 07fb45dfacf5147fa3eba6ffade3f64ce4a69741
Parents: 138912c
Author: Steve Loughran <st...@apache.org>
Authored: Fri Nov 6 12:34:45 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Nov 6 13:01:24 2015 +0000

----------------------------------------------------------------------
 .../apache/slider/providers/ProviderRole.java   |  25 +++--
 .../slideram/SliderAMClientProvider.java        |   2 +-
 .../slider/server/appmaster/state/AppState.java | 111 +++++++++----------
 .../appmaster/state/OutstandingRequest.java     |  15 ++-
 .../server/appmaster/state/RoleHistory.java     |  23 +---
 .../server/appmaster/state/RoleStatus.java      |  14 ++-
 .../resources/AggregateConfResource.java        |  26 ++---
 .../appstate/TestMockAppStateAAPlacement.groovy |   2 +-
 ...tRoleHistoryOutstandingRequestTracker.groovy |  51 ++++++---
 .../model/history/TestRoleHistoryRW.groovy      |   4 +-
 .../TestRoleHistoryRequestTracking.groovy       |  10 +-
 .../model/mock/BaseMockAppStateTest.groovy      |  10 +-
 .../appmaster/model/mock/MockFactory.groovy     |   9 +-
 13 files changed, 152 insertions(+), 150 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/07fb45df/slider-core/src/main/java/org/apache/slider/providers/ProviderRole.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/ProviderRole.java b/slider-core/src/main/java/org/apache/slider/providers/ProviderRole.java
index 3009f50..1b95b42 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/ProviderRole.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/ProviderRole.java
@@ -32,13 +32,15 @@ public final class ProviderRole {
   public int placementPolicy;
   public int nodeFailureThreshold;
   public final long placementTimeoutSeconds;
+  public final String labelExpression;
 
   public ProviderRole(String name, int id) {
     this(name,
         id,
         PlacementPolicy.DEFAULT,
         ResourceKeys.DEFAULT_NODE_FAILURE_THRESHOLD,
-        ResourceKeys.DEFAULT_PLACEMENT_ESCALATE_DELAY_SECONDS);
+        ResourceKeys.DEFAULT_PLACEMENT_ESCALATE_DELAY_SECONDS,
+        ResourceKeys.DEF_YARN_LABEL_EXPRESSION);
   }
 
   /**
@@ -49,18 +51,20 @@ public final class ProviderRole {
    * @param nodeFailureThreshold threshold for node failures (within a reset interval)
    * after which a node failure is considered an app failure
    * @param placementTimeoutSeconds for lax placement, timeout in seconds before
-   * a relaxed placement request is generated.
+   * @param labelExpression label expression for requests; may be null
    */
   public ProviderRole(String name,
       int id,
       int policy,
       int nodeFailureThreshold,
-      long placementTimeoutSeconds) {
+      long placementTimeoutSeconds,
+      String labelExpression) {
     this.name = name;
     this.id = id;
     this.placementPolicy = policy;
     this.nodeFailureThreshold = nodeFailureThreshold;
     this.placementTimeoutSeconds = placementTimeoutSeconds;
+    this.labelExpression = labelExpression;
   }
 
   @Override
@@ -83,11 +87,14 @@ public final class ProviderRole {
 
   @Override
   public String toString() {
-    return "ProviderRole {" +
-           "name='" + name + '\'' +
-           ", id=" + id +
-           ", policy=" + placementPolicy +
-           ", nodeFailureThreshold=" + nodeFailureThreshold +
-           '}';
+    final StringBuilder sb = new StringBuilder("ProviderRole{");
+    sb.append("name='").append(name).append('\'');
+    sb.append(", id=").append(id);
+    sb.append(", placementPolicy=").append(placementPolicy);
+    sb.append(", nodeFailureThreshold=").append(nodeFailureThreshold);
+    sb.append(", placementTimeoutSeconds=").append(placementTimeoutSeconds);
+    sb.append(", labelExpression='").append(labelExpression).append('\'');
+    sb.append('}');
+    return sb.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/07fb45df/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java b/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
index 9bd4dc9..3be0f48 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
@@ -90,7 +90,7 @@ public class SliderAMClientProvider extends AbstractClientProvider
       new ProviderRole(COMPONENT_AM, KEY_AM,
           PlacementPolicy.EXCLUDE_FROM_FLEXING,
           ResourceKeys.DEFAULT_NODE_FAILURE_THRESHOLD, 
-          0);
+          0, "");
 
   /**
    * Initialize role list

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/07fb45df/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 e47ef34..946d45f 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
@@ -22,7 +22,6 @@ import com.codahale.metrics.Counter;
 import com.codahale.metrics.MetricRegistry;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -141,7 +140,7 @@ public class AppState {
   private ConfTreeOperations resourcesSnapshot;
   private ConfTreeOperations appConfSnapshot;
   private ConfTreeOperations internalsSnapshot;
-  
+
   /**
    * This is the status, the live model
    */
@@ -151,7 +150,7 @@ public class AppState {
    * Metadata provided by the AM for use in filling in status requests
    */
   private Map<String, String> applicationInfo;
-  
+
   /**
    * Client properties created via the provider -static for the life
    * of the application
@@ -278,10 +277,10 @@ public class AppState {
   private RoleHistory roleHistory;
   private Configuration publishedProviderConf;
   private long startTimeThreshold;
-  
+
   private int failureThreshold = 10;
   private int nodeFailureThreshold = 3;
-  
+
   private String logServerURL = "";
 
   /**
@@ -618,7 +617,7 @@ public class AppState {
         ResourceKeys.COMPONENT_PLACEMENT_POLICY,
         Integer.toString(PlacementPolicy.DEFAULT));
     int placement = SliderUtils.parseAndValidate("value of " + name + " " +
-                                                 ResourceKeys.COMPONENT_PLACEMENT_POLICY,
+        ResourceKeys.COMPONENT_PLACEMENT_POLICY,
         placementOpt, 0, 0, -1);
     int placementTimeout =
         component.getOptionInt(ResourceKeys.PLACEMENT_ESCALATE_DELAY,
@@ -627,7 +626,8 @@ public class AppState {
         priority,
         placement,
         getNodeFailureThresholdForRole(name),
-        placementTimeout);
+        placementTimeout,
+        component.getOption(YARN_LABEL_EXPRESSION, DEF_YARN_LABEL_EXPRESSION));
     log.info("New {} ", newRole);
     return newRole;
   }
@@ -653,7 +653,7 @@ public class AppState {
     log.debug("Instance definition updated");
     //note the time 
     snapshotTime = now();
-    
+
     // resolve references if not already done
     instanceDefinition.resolve();
 
@@ -682,15 +682,14 @@ public class AppState {
     clusterStatusTemplate =
       ClusterDescriptionOperations.buildFromInstanceDefinition(
           instanceDefinition);
-    
 
 //     Add the -site configuration properties
     for (Map.Entry<String, String> prop : clientProperties.entrySet()) {
       clusterStatusTemplate.clientProperties.put(prop.getKey(), prop.getValue());
     }
-    
+
   }
-  
+
   /**
    * The resource configuration is updated -review and update state.
    * @param resources updated resources specification
@@ -723,9 +722,9 @@ public class AppState {
   private List<ProviderRole> buildRoleRequirementsFromResources() throws BadConfigException {
 
     List<ProviderRole> newRoles = new ArrayList<>(0);
-    
-    //now update every role's desired count.
-    //if there are no instance values, that role count goes to zero
+
+    // now update every role's desired count.
+    // if there are no instance values, that role count goes to zero
 
     ConfTreeOperations resources =
         instanceDefinition.getResourceOperations();
@@ -751,8 +750,8 @@ public class AppState {
       }
     }
 
-    //now the dynamic ones. Iterate through the the cluster spec and
-    //add any role status entries not in the role status
+    // now the dynamic ones. Iterate through the the cluster spec and
+    // add any role status entries not in the role status
     Set<String> roleNames = resources.getComponentNames();
     for (String name : roleNames) {
       if (!roles.containsKey(name)) {
@@ -803,7 +802,7 @@ public class AppState {
    * @throws BadConfigException if a role of that priority already exists
    */
   public RoleStatus buildRole(ProviderRole providerRole) throws BadConfigException {
-    //build role status map
+    // build role status map
     int priority = providerRole.id;
     if (roleStatusMap.containsKey(priority)) {
       throw new BadConfigException("Duplicate Provider Key: %s and %s",
@@ -843,7 +842,7 @@ public class AppState {
     //it is also added to the set of live nodes
     getLiveContainers().put(containerId, am);
     putOwnedContainer(containerId, am);
-    
+
     // patch up the role status
     RoleStatus roleStatus = roleStatusMap.get(
         (SliderKeys.ROLE_AM_PRIORITY_INDEX));
@@ -876,7 +875,12 @@ public class AppState {
     return appMasterNode;
   }
 
-
+  /**
+   * Look up the status entry of a role or raise an exception
+   * @param key role ID
+   * @return the status entry
+   * @throws RuntimeException if the role cannot be found
+   */
   public RoleStatus lookupRoleStatus(int key) {
     RoleStatus rs = getRoleStatusMap().get(key);
     if (rs == null) {
@@ -884,8 +888,15 @@ public class AppState {
     }
     return rs;
   }
-  
-  public RoleStatus lookupRoleStatus(Container c) throws YarnRuntimeException {
+
+  /**
+   * Look up the status entry of a container or raise an exception
+   *
+   * @param c container
+   * @return the status entry
+   * @throws RuntimeException if the role cannot be found
+   */
+  public RoleStatus lookupRoleStatus(Container c) {
     return lookupRoleStatus(ContainerPriority.extractRole(c));
   }
 
@@ -1093,8 +1104,7 @@ public class AppState {
     }
     return map;
   }
-  
-  
+
   /**
    * Build a map of role->nodename->node-info
    * 
@@ -1173,10 +1183,10 @@ public class AppState {
         RoleStatus role,
         Resource capability) {
     buildResourceRequirements(role, capability);
-    String labelExpression = getLabelExpression(role);
+    String labelExpression = role.getLabelExpression();
     //get the role history to select a suitable node, if available
     AMRMClient.ContainerRequest containerRequest =
-      createContainerRequest(role, capability, labelExpression);
+      createContainerRequest(role, capability);
     return  containerRequest;
   }
 
@@ -1184,18 +1194,15 @@ public class AppState {
    * Create a container request.
    * Update internal state, such as the role request count
    * This is where role history information will be used for placement decisions -
+   * @param labelExpression label expression to satisfy
    * @param role role
    * @param resource requirements
-   * @param labelExpression label expression to satisfy
    * @return the container request to submit
    */
   private AMRMClient.ContainerRequest createContainerRequest(RoleStatus role,
-                                                            Resource resource,
-                                                            String labelExpression) {
-    
-    
+      Resource resource) {
     AMRMClient.ContainerRequest request;
-    request = roleHistory.requestNode(role, resource, labelExpression);
+    request = roleHistory.requestNode(role, resource);
     incrementRequestCount(role);
     return request;
   }
@@ -1211,7 +1218,6 @@ public class AppState {
     incOutstandingContainerRequests();
   }
 
-
   /**
    * dec requested count of a role
    * <p>
@@ -1261,7 +1267,7 @@ public class AppState {
                                      String option,
                                      int defVal,
                                      int maxVal) {
-    
+
     String val = resources.getComponentOpt(name, option,
         Integer.toString(defVal));
     Integer intVal;
@@ -1273,7 +1279,6 @@ public class AppState {
     return intVal;
   }
 
-  
   /**
    * Build up the resource requirements for this role from the
    * cluster specification, including substituing max allowed values
@@ -1299,17 +1304,6 @@ public class AppState {
   }
 
   /**
-   * Extract the label expression for this role.
-   * @param role role
-   */
-  public String getLabelExpression(RoleStatus role) {
-    // Set up resource requirements from role values
-    String name = role.getName();
-    ConfTreeOperations resources = getResourcesSnapshot();
-    return resources.getComponentOpt(name, YARN_LABEL_EXPRESSION, DEF_YARN_LABEL_EXPRESSION);
-  }
-
-  /**
    * add a launched container to the node map for status responses
    * @param container id
    * @param node node details
@@ -1471,10 +1465,10 @@ public class AppState {
       return sb.toString();
     }
   }
-  
+
   /**
    * handle completed node in the CD -move something from the live
-   * server list to the completed server list
+   * server list to the completed server list.
    * @param status the node that has just completed
    * @return NodeCompletionResult
    */
@@ -1534,7 +1528,7 @@ public class AppState {
           if (failedContainer != null) {
             String completedLogsUrl = getLogsURLForContainer(failedContainer);
             message = String.format("Failure %s on host %s (%d): %s",
-                roleInstance.getContainerId().toString(),
+                roleInstance.getContainerId(),
                 failedContainer.getNodeId().getHost(),
                 exitStatus,
                 completedLogsUrl);
@@ -1583,10 +1577,10 @@ public class AppState {
       log.warn("Received notification of completion of unknown node {}", id);
       completionOfNodeNotInLiveListEvent.incrementAndGet();
     }
-    
+
     // and the active node list if present
     removeOwnedContainer(containerId);
-    
+
     // finally, verify the node doesn't exist any more
     assert !containersBeingReleased.containsKey(
         containerId) : "container still in release queue";
@@ -1621,9 +1615,6 @@ public class AppState {
     return completedLogsUrl;
   }
 
-
-  
-  
   /**
    * Return the percentage done that Slider is to have YARN display in its
    * Web UI
@@ -1652,7 +1643,7 @@ public class AppState {
   public ClusterDescription refreshClusterStatus() {
     return refreshClusterStatus(null);
   }
-  
+
   /**
    * Update the cluster description with the current application state
    * @param providerStatus status from the provider for the cluster info section
@@ -1702,13 +1693,12 @@ public class AppState {
       cd.statistics.put(rolename, stats);
     }
 
-    
     Map<String, Integer> sliderstats = getLiveStatistics();
     cd.statistics.put(SliderKeys.COMPONENT_AM, sliderstats);
-    
+
     // liveness
     cd.liveness = getApplicationLivenessInformation();
-    
+
     return cd;
   }
 
@@ -1723,7 +1713,7 @@ public class AppState {
     li.allRequestsSatisfied = outstanding <= 0;
     return li;
   }
-  
+
   /**
    * Get the live statistics map
    * @return a map of statistics values, defined in the {@link StatusKeys}
@@ -1861,7 +1851,7 @@ public class AppState {
   public List<AbstractRMOperation> escalateOutstandingRequests() {
     return roleHistory.escalateOutstandingRequests();
   }
-  
+
   /**
    * Look at the allocation status of one role, and trigger add/release
    * actions if the number of desired role instances doesn't equal 
@@ -2103,7 +2093,6 @@ public class AppState {
       //get the role
       final ContainerId cid = container.getId();
       final RoleStatus role = lookupRoleStatus(container);
-      
 
       //dec requested count
       decrementRequestCount(role);
@@ -2213,7 +2202,7 @@ public class AppState {
              cid,
              roleName,
              containerHostInfo);
-    
+
     //update app state internal structures and maps
 
     RoleInstance instance = new RoleInstance(container);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/07fb45df/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java
index 85bd259..38bc96f 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java
@@ -71,21 +71,21 @@ public final class OutstandingRequest {
   /**
    * Requested time in millis.
    * <p>
-   * Only valid after {@link #buildContainerRequest(Resource, RoleStatus, long, String)}
+   * Only valid after {@link #buildContainerRequest(Resource, RoleStatus, long)}
    */
   private AMRMClient.ContainerRequest issuedRequest;
   
   /**
    * Requested time in millis.
    * <p>
-   * Only valid after {@link #buildContainerRequest(Resource, RoleStatus, long, String)}
+   * Only valid after {@link #buildContainerRequest(Resource, RoleStatus, long)}
    */
   private long requestedTimeMillis;
 
   /**
    * Time in millis after which escalation should be triggered..
    * <p>
-   * Only valid after {@link #buildContainerRequest(Resource, RoleStatus, long, String)}
+   * Only valid after {@link #buildContainerRequest(Resource, RoleStatus, long)}
    */
   private long escalationTimeoutMillis;
 
@@ -178,16 +178,15 @@ public final class OutstandingRequest {
    * @param resource resource
    * @param role role
    * @param time time in millis to record as request time
-   * @param labelExpression label to satisfy
    * @return the request to raise
    */
   public synchronized AMRMClient.ContainerRequest buildContainerRequest(
-      Resource resource, RoleStatus role, long time, String labelExpression) {
+      Resource resource, RoleStatus role, long time) {
     Preconditions.checkArgument(resource != null, "null `resource` arg");
     Preconditions.checkArgument(role != null, "null `role` arg");
 
     // cache label for escalation
-    label = labelExpression;
+    label = role.getLabelExpression();
     requestedTimeMillis = time;
     escalationTimeoutMillis = time + role.getPlacementTimeoutSeconds() * 1000;
     String[] hosts;
@@ -218,7 +217,7 @@ public final class OutstandingRequest {
       escalated = true;
       // and forbid it happening
       mayEscalate = false;
-      nodeLabels = labelExpression;
+      nodeLabels = label;
     }
     Priority pri = ContainerPriority.createPriority(roleId, !relaxLocality);
     priority = pri.getPriority();
@@ -254,7 +253,7 @@ public final class OutstandingRequest {
 
     String[] nodes;
     List<String> issuedRequestNodes = issuedRequest.getNodes();
-    if (label == null && issuedRequestNodes != null) {
+    if (SliderUtils.isUnset(label) && issuedRequestNodes != null) {
       nodes = issuedRequestNodes.toArray(new String[issuedRequestNodes.size()]);
     } else {
       nodes = null;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/07fb45df/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
index a0aa3bc..34340a2 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
@@ -611,31 +611,14 @@ public class RoleHistory {
    * Returns the request that is now being tracked.
    * If the node instance is not null, it's details about the role is incremented
    *
-   *
    * @param node node to target or null for "any"
    * @param role role to request
-   * @param labelExpression label to satisfy
    * @return the container priority
    */
   public synchronized AMRMClient.ContainerRequest requestInstanceOnNode(
-    NodeInstance node, RoleStatus role, Resource resource, String labelExpression) {
+      NodeInstance node, RoleStatus role, Resource resource) {
     OutstandingRequest outstanding = outstandingRequests.newRequest(node, role.getKey());
-    return outstanding.buildContainerRequest(resource, role, now(), labelExpression);
-  }
-
-  /**
-   * Find a node for a role and request an instance on that (or a location-less
-   * instance) with a label expression
-   * @param role role status
-   * @param resource resource capabilities
-   * @param labelExpression label to satisfy
-   * @return a request ready to go
-   */
-  public synchronized AMRMClient.ContainerRequest requestNode(RoleStatus role,
-                                                              Resource resource,
-                                                              String labelExpression) {
-    NodeInstance node = findNodeForNewInstance(role);
-    return requestInstanceOnNode(node, role, resource, labelExpression);
+    return outstanding.buildContainerRequest(resource, role, now());
   }
 
   /**
@@ -648,7 +631,7 @@ public class RoleHistory {
   public synchronized AMRMClient.ContainerRequest requestNode(RoleStatus role,
                                                               Resource resource) {
     NodeInstance node = findNodeForNewInstance(role);
-    return requestInstanceOnNode(node, role, resource, null);
+    return requestInstanceOnNode(node, role, resource);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/07fb45df/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
index 52df406..20f5802 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
@@ -41,7 +41,6 @@ public final class RoleStatus implements Cloneable {
    * Role priority
    */
   private final int key;
-
   private final ProviderRole providerRole;
 
   private int desired, actual, requested, releasing;
@@ -315,7 +314,7 @@ public final class RoleStatus implements Cloneable {
            ", actual=" + actual +
            ", requested=" + requested +
            ", releasing=" + releasing +
-           ",  pendingAntiAffineRequestCount=" + pendingAntiAffineRequestCount +
+           ", pendingAntiAffineRequestCount=" + pendingAntiAffineRequestCount +
            ", failed=" + failed +
            ", failed recently=" + failedRecently.get() +
            ", node failed=" + nodeFailed.get() +
@@ -324,6 +323,7 @@ public final class RoleStatus implements Cloneable {
            ", startFailed=" + startFailed +
            ", completed=" + completed +
            ", failureMessage='" + failureMessage + '\'' +
+           ", providerRole=" + providerRole +
            '}';
   }
 
@@ -373,7 +373,15 @@ public final class RoleStatus implements Cloneable {
     info.pendingAntiAffineRequestCount = pendingAntiAffineRequestCount;
     return info;
   }
-  
+
+  /**
+   * Get the (possibly null) label expression for this role
+   * @return a string or null
+   */
+  public String getLabelExpression() {
+    return providerRole.labelExpression;
+  }
+
   /**
    * Compare two role status entries by name
    */

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/07fb45df/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java
index 75d417b..ebffd1a 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java
@@ -37,23 +37,13 @@ public class AggregateConfResource {
 
   public AggregateConfResource(AggregateConf conf, UriBuilder uriBuilder) {
     if (uriBuilder != null) {
-      this.href =
-          uriBuilder.build().toASCIIString();
-      resources =
-          ResourceFactory.createConfTreeResource(conf.getAppConf(),
-                                                 uriBuilder.clone().path(
-                                                     "configurations").path(
-                                                     "resources"));
-      internal =
-          ResourceFactory.createConfTreeResource(conf.getInternal(),
-                                                 uriBuilder.clone().path(
-                                                     "configurations").path(
-                                                     "internal"));
-      appConf =
-          ResourceFactory.createConfTreeResource(conf.getAppConf(),
-                                                 uriBuilder.clone().path(
-                                                     "configurations").path(
-                                                     "appConf"));
+      this.href = uriBuilder.build().toASCIIString();
+      resources = ResourceFactory.createConfTreeResource(conf.getAppConf(),
+                   uriBuilder.clone().path("configurations").path("resources"));
+      internal = ResourceFactory.createConfTreeResource(conf.getInternal(),
+                   uriBuilder.clone().path("configurations").path("internal"));
+      appConf = ResourceFactory.createConfTreeResource(conf.getAppConf(),
+                   uriBuilder.clone().path("configurations").path("appConf"));
       initConfMap();
     } else {
       resources = null;
@@ -63,7 +53,7 @@ public class AggregateConfResource {
   }
 
   private void initConfMap() {
-    confMap = new HashMap<String, ConfTreeResource>();
+    confMap = new HashMap<>();
     confMap.put("internal", internal);
     confMap.put("resources", resources);
     confMap.put("appConf", appConf);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/07fb45df/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 0e9fad0..6168146 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
@@ -44,7 +44,7 @@ import static org.apache.slider.server.appmaster.state.ContainerPriority.extract
 class TestMockAppStateAAPlacement extends BaseMockAppStateTest
     implements MockRoles {
 
-  @Test
+//  @Test
   public void testAllocateAA() throws Throwable {
 
     def aaRole = role2Status

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/07fb45df/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
index 8399d53..745d40f 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
@@ -27,6 +27,7 @@ import org.apache.slider.server.appmaster.model.mock.MockResource
 import org.apache.slider.server.appmaster.operations.AbstractRMOperation
 import org.apache.slider.server.appmaster.operations.CancelSingleRequest
 import org.apache.slider.server.appmaster.operations.ContainerRequestOperation
+import org.apache.slider.server.appmaster.state.AppStateBindingInfo
 import org.apache.slider.server.appmaster.state.ContainerAllocationOutcome
 import org.apache.slider.server.appmaster.state.ContainerPriority
 import org.apache.slider.server.appmaster.state.NodeInstance
@@ -37,12 +38,28 @@ import org.junit.Test
 
 class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
 
+  public static final String WORKERS_LABEL = "workers"
   NodeInstance host1 = new NodeInstance("host1", 3)
   NodeInstance host2 = new NodeInstance("host2", 3)
   def resource = factory.newResource(48, 1)
 
   OutstandingRequestTracker tracker = new OutstandingRequestTracker()
 
+  public static final ProviderRole WORKER = new ProviderRole(
+      "worker",
+      5,
+      PlacementPolicy.NONE,
+      2,
+      1,
+      WORKERS_LABEL)
+
+  @Override
+  AppStateBindingInfo buildBindingInfo() {
+    def bindingInfo = super.buildBindingInfo()
+    bindingInfo.roles = [ WORKER ] + bindingInfo.roles
+    bindingInfo
+  }
+
   @Test
   public void testAddRetrieveEntry() throws Throwable {
     OutstandingRequest request = tracker.newRequest(host1, 0)
@@ -54,10 +71,10 @@ class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
   @Test
   public void testAddCompleteEntry() throws Throwable {
     def req1 = tracker.newRequest(host1, 0)
-    req1.buildContainerRequest(resource, role0Status, 0, "")
+    req1.buildContainerRequest(resource, role0Status, 0)
 
-    tracker.newRequest(host2, 0).buildContainerRequest(resource, role0Status, 0, "")
-    tracker.newRequest(host1, 1).buildContainerRequest(resource, role0Status, 0, "")
+    tracker.newRequest(host2, 0).buildContainerRequest(resource, role0Status, 0)
+    tracker.newRequest(host1, 1).buildContainerRequest(resource, role0Status, 0)
 
     def allocation = tracker.onContainerAllocated(1, "host1", null)
     assert allocation.outcome == ContainerAllocationOutcome.Placed
@@ -82,7 +99,7 @@ class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
   @Test
   public void testRemoveOpenRequestUnissued() throws Throwable {
     def req1 = tracker.newRequest(null, 0)
-    req1.buildContainerRequest(resource, role0Status, 0, "")
+    req1.buildContainerRequest(resource, role0Status, 0)
     assert tracker.listOpenRequests().size() == 1
     def c1 = factory.newContainer(null, new MockPriority(0))
     c1.resource = resource
@@ -97,7 +114,7 @@ class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
   @Test
   public void testIssuedOpenRequest() throws Throwable {
     def req1 = tracker.newRequest(null, 0)
-    req1.buildContainerRequest(resource, role0Status, 0, "")
+    req1.buildContainerRequest(resource, role0Status, 0)
     assert tracker.listOpenRequests().size() == 1
 
     def pri = ContainerPriority.buildPriority(0, false)
@@ -142,7 +159,7 @@ class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
     // first request: default placement
     assert role0Status.placementPolicy == PlacementPolicy.DEFAULT;
     final def (res0, outstanding0) = newRequest(role0Status)
-    final def initialRequest = outstanding0.buildContainerRequest(res0, role0Status, 0, null)
+    final def initialRequest = outstanding0.buildContainerRequest(res0, role0Status, 0)
     assert outstanding0.issuedRequest != null;
     assert outstanding0.located
     assert !outstanding0.escalated
@@ -178,7 +195,7 @@ class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
     // build that second request from an anti-affine entry
     // these get placed as well
     now += interval
-    final def containerReq2 = outstanding2.buildContainerRequest(res2, role2Status, now, null)
+    final def containerReq2 = outstanding2.buildContainerRequest(res2, role2Status, now)
     // escalate a little bit more
     final List<AbstractRMOperation> escalations2 = tracker.escalateOutstandingRequests(now)
     // and expect no new entries
@@ -219,18 +236,18 @@ class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
     resource.virtualCores = 1
     resource.memory = 48;
 
-    def label = "workers"
+    def workerRole = lookupRole(WORKER.name)
     // initial request
-    def yarnRequest = req1.buildContainerRequest(resource, role0Status, 0, label)
+    def yarnRequest = req1.buildContainerRequest(resource, workerRole, 0)
     assert (yarnRequest.nodeLabelExpression == null)
     assert (!yarnRequest.relaxLocality)
     def yarnRequest2 = req1.escalate()
-    assert (yarnRequest2.nodeLabelExpression == label)
+    assert (yarnRequest2.nodeLabelExpression == WORKERS_LABEL)
     assert (yarnRequest2.relaxLocality)
   }
 
   /**
-   * If the placement doesnt include a lablel, then the escalation request
+   * If the placement doesnt include a label, then the escalation request
    * retains the node list, but sets relaxLocality==true
    * @throws Throwable
    */
@@ -244,13 +261,13 @@ class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
 
     def label = null
     // initial request
-    def yarnRequest = req1.buildContainerRequest(resource, role0Status, 0, label)
-    assert (yarnRequest.nodes != null)
-    assert (yarnRequest.nodeLabelExpression == null)
-    assert (!yarnRequest.relaxLocality)
+    def yarnRequest = req1.buildContainerRequest(resource, role0Status, 0)
+    assert yarnRequest.nodes != null
+    assert !yarnRequest.nodeLabelExpression
+    assert !yarnRequest.relaxLocality
     def yarnRequest2 = req1.escalate()
-    assert (yarnRequest2.nodes != null)
-    assert (yarnRequest2.relaxLocality)
+    assert yarnRequest2.nodes != null
+    assert yarnRequest2.relaxLocality
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/07fb45df/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
index a1e424f..254c0b6 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
@@ -22,6 +22,7 @@ import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.fs.FSDataOutputStream
 import org.apache.hadoop.fs.Path
+import org.apache.slider.api.ResourceKeys
 import org.apache.slider.providers.PlacementPolicy
 import org.apache.slider.providers.ProviderRole
 import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
@@ -46,7 +47,8 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
       3,
       PlacementPolicy.STRICT,
       3,
-      3)
+      3,
+      ResourceKeys.DEF_YARN_LABEL_EXPRESSION)
 
   @Override
   String getTestName() {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/07fb45df/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
index 42d0c50..c6dcb07 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
@@ -86,8 +86,8 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
     assertListEquals([age2Active0], roleHistory.cloneRecentNodeList(0))
     roleHistory.requestInstanceOnNode(ni,
         roleStatus,
-        resource,
-        "")
+        resource
+    )
   }
 
   @Test
@@ -106,8 +106,8 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
     // which is translated to a no-location request
     AMRMClient.ContainerRequest req = roleHistory.requestInstanceOnNode(ni,
         roleStatus,
-        resource,
-        "")
+        resource
+    )
 
     assertNull(req.nodes)
 
@@ -120,7 +120,7 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
     // looking for a node should now find one
     ni = roleHistory.findNodeForNewInstance(roleStatus)
     assert ni == age3Active0
-    req = roleHistory.requestInstanceOnNode(ni, roleStatus, resource, "")
+    req = roleHistory.requestInstanceOnNode(ni, roleStatus, resource)
     assert 1 == req.nodes.size()
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/07fb45df/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
index a065518..40d7fd7 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
@@ -128,15 +128,19 @@ abstract class BaseMockAppStateTest extends SliderTestBase implements MockRoles
   }
 
   public RoleStatus getRole0Status() {
-    return appState.lookupRoleStatus(ROLE0)
+    lookupRole(ROLE0)
+  }
+
+  public RoleStatus lookupRole(String role) {
+    appState.lookupRoleStatus(role)
   }
 
   public RoleStatus getRole1Status() {
-    return appState.lookupRoleStatus(ROLE1)
+    lookupRole(ROLE1)
   }
 
   public RoleStatus getRole2Status() {
-    return appState.lookupRoleStatus(ROLE2)
+    lookupRole(ROLE2)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/07fb45df/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 f7fd641..0a4a93e 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
@@ -49,21 +49,24 @@ class MockFactory implements MockRoles {
       0,
       PlacementPolicy.DEFAULT,
       2,
-      1)
+      1,
+      ResourceKeys.DEF_YARN_LABEL_EXPRESSION)
   // role 1 is strict. timeout should be irrelevant; same as failures
   public static final ProviderRole PROVIDER_ROLE1 = new ProviderRole(
       MockRoles.ROLE1,
       1,
       PlacementPolicy.STRICT,
       2,
-      1)
+      1,
+      ResourceKeys.DEF_YARN_LABEL_EXPRESSION)
   // role 2: longer delay and anti-affinity
   public static final ProviderRole PROVIDER_ROLE2 = new ProviderRole(
       MockRoles.ROLE2,
       2,
       PlacementPolicy.ANTI_AFFINITY_REQUIRED,
       2,
-      2)
+      2,
+      ResourceKeys.DEF_YARN_LABEL_EXPRESSION)
 
   int appIdCount;
   int attemptIdCount;



[14/22] incubator-slider git commit: SLIDER-964 AggregateConfResource exports appconf under internal/

Posted by st...@apache.org.
SLIDER-964 AggregateConfResource exports appconf under internal/


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: ca43d1bea353fe15057d24e71e569d43d2a98c1a
Parents: 07fb45d
Author: Steve Loughran <st...@apache.org>
Authored: Fri Nov 6 13:57:19 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Nov 6 13:57:19 2015 +0000

----------------------------------------------------------------------
 .../web/rest/management/TestAMManagementWebServices.java        | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ca43d1be/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
index 70a6bcb..44eb692 100644
--- a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
+++ b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
@@ -114,9 +114,10 @@ public class TestAMManagementWebServices extends JerseyTest {
       try {
         JsonSerDeser<ConfTree> confTreeJsonSerDeser = new JsonSerDeser<>(ConfTree.class);
         AggregateConf aggregateConf = new AggregateConf(
-            confTreeJsonSerDeser.fromResource(EXAMPLES + "internal.json"),
+            confTreeJsonSerDeser.fromResource(EXAMPLES + "resources.json"),
             confTreeJsonSerDeser.fromResource(EXAMPLES + "app_configuration.json"),
-            confTreeJsonSerDeser.fromResource(EXAMPLES + "resources.json"));
+            confTreeJsonSerDeser.fromResource(EXAMPLES + "internal.json")
+            );
         aggregateConf.setName("test");
         return aggregateConf;
       } catch (IOException e) {


[04/22] incubator-slider git commit: SLIDER-82 preparing ground for anti-affinity. * app state binding moves from multiple args to a single binding struct; this significantly simplifies test setup * references in roleHistory to available nodes are replac

Posted by st...@apache.org.
SLIDER-82 preparing ground for anti-affinity.
* app state binding moves from multiple args to a single binding struct; this significantly simplifies test setup
* references in roleHistory to available nodes are replace with "recent", as that is what they currently are


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: f2f2c766b887482ca4a6940e9c11e21f58660d9a
Parents: a66f7db
Author: Steve Loughran <st...@apache.org>
Authored: Wed Nov 4 16:35:03 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Nov 5 13:19:03 2015 +0000

----------------------------------------------------------------------
 .../slider/client/SliderYarnClientImpl.java     |  9 +-
 .../slider/providers/PlacementPolicy.java       |  7 +-
 .../server/appmaster/SliderAppMaster.java       | 93 ++++++++++----------
 .../slider/server/appmaster/state/AppState.java | 81 +++++++----------
 .../appmaster/state/AppStateBindingInfo.java    | 59 +++++++++++++
 .../server/appmaster/state/RoleHistory.java     | 79 +++++++++--------
 .../TestMockAppStateDynamicHistory.groovy       | 28 +-----
 .../TestMockAppStateDynamicRoles.groovy         | 33 +++----
 .../TestMockAppStateFlexDynamicRoles.groovy     | 44 ++++-----
 .../TestMockAppStateRebuildOnAMRestart.groovy   | 32 +++----
 .../TestMockContainerResourceAllocations.groovy |  5 +-
 .../TestRoleHistoryContainerEvents.groovy       |  2 +-
 ...stRoleHistoryFindNodesForNewInstances.groovy |  2 +-
 .../model/history/TestRoleHistoryRW.groovy      |  4 +-
 .../TestRoleHistoryRequestTracking.groovy       |  4 +-
 .../model/mock/BaseMockAppStateTest.groovy      | 41 +++++----
 .../appmaster/model/mock/MockAppState.groovy    | 10 ++-
 .../web/rest/agent/TestAMAgentWebServices.java  | 47 ++++------
 .../management/TestAMManagementWebServices.java | 71 +++++----------
 19 files changed, 302 insertions(+), 349 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java b/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
index 3b7a65c..42759fd 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
@@ -55,8 +55,7 @@ import java.util.Set;
  * from the slider entry point service
  */
 public class SliderYarnClientImpl extends YarnClientImpl {
-  protected static final Logger
-    log = LoggerFactory.getLogger(SliderYarnClientImpl.class);
+  protected static final Logger log = LoggerFactory.getLogger(SliderYarnClientImpl.class);
 
   /**
    * Keyword to use in the {@link #emergencyForceKill(String)}
@@ -96,10 +95,10 @@ public class SliderYarnClientImpl extends YarnClientImpl {
   public List<ApplicationReport> listDeployedInstances(String user)
     throws YarnException, IOException {
     Preconditions.checkArgument(user != null, "Null User");
-    Set<String> types = new HashSet<String>(1);
+    Set<String> types = new HashSet<>(1);
     types.add(SliderKeys.APP_TYPE);
     List<ApplicationReport> allApps = getApplications(types);
-    List<ApplicationReport> results = new ArrayList<ApplicationReport>();
+    List<ApplicationReport> results = new ArrayList<>();
     for (ApplicationReport report : allApps) {
       if (StringUtils.isEmpty(user) || user.equals(report.getUser())) {
         results.add(report);
@@ -330,8 +329,6 @@ public class SliderYarnClientImpl extends YarnClientImpl {
     Preconditions.checkArgument(StringUtils.isNotEmpty(appname),
         "Null/empty application name");
     Preconditions.checkArgument(desiredState != null, "Null desiredState");
-    ApplicationReport found = null;
-    ApplicationReport foundAndLive = null;
     log.debug("Searching {} records for instance name {} in state '{}'",
         instances.size(), appname, desiredState);
     for (ApplicationReport app : instances) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/main/java/org/apache/slider/providers/PlacementPolicy.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/PlacementPolicy.java b/slider-core/src/main/java/org/apache/slider/providers/PlacementPolicy.java
index 4e85a93..e0913a5 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/PlacementPolicy.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/PlacementPolicy.java
@@ -27,7 +27,12 @@ public class PlacementPolicy {
   /**
    * Default value: history used, anti-affinity hinted at on rebuild/flex up
    */
-  public static final int DEFAULT = 0;
+  public static final int NONE = 0;
+
+  /**
+   * Default value: history used, anti-affinity hinted at on rebuild/flex up
+   */
+  public static final int DEFAULT = NONE;
 
   /**
    * Strict placement: when asking for an instance for which there is

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/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 df91d7f..b552290 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
@@ -59,13 +59,16 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
 import org.apache.hadoop.yarn.client.api.async.NMClientAsync;
 import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.*;
 import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.registry.client.api.RegistryOperations;
 import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
@@ -84,6 +87,7 @@ import org.apache.slider.api.ResourceKeys;
 import org.apache.slider.api.RoleKeys;
 import org.apache.slider.api.StatusKeys;
 import org.apache.slider.api.proto.SliderClusterAPI;
+import org.apache.slider.client.SliderYarnClientImpl;
 import org.apache.slider.common.SliderExitCodes;
 import org.apache.slider.common.SliderKeys;
 import org.apache.slider.common.params.AbstractActionArgs;
@@ -143,6 +147,7 @@ import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
 import org.apache.slider.server.appmaster.rpc.SliderIPCService;
 import org.apache.slider.server.appmaster.security.SecurityConfiguration;
 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.ProviderAppState;
 import org.apache.slider.server.appmaster.operations.RMOperationHandler;
@@ -356,16 +361,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   private RegistryOperations registryOperations;
 
   /**
-   * Record of the max no. of cores allowed in this cluster
-   */
-  private int containerMaxCores;
-
-  /**
-   * limit container memory
-   */
-  private int containerMaxMemory;
-
-  /**
    * The stop request received...the exit details are extracted
    * from this
    */
@@ -415,6 +410,12 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
    */
   private boolean securityEnabled;
   private ContentCache contentCache;
+  private SliderYarnClientImpl yarnClient;
+
+  /**
+   * resource limits
+   */
+  private Resource maximumResourceCapability;
 
   /**
    * Service Constructor
@@ -437,10 +438,8 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     // Load in the server configuration - if it is actually on the Classpath
     URL serverXmlUrl = ConfigHelper.getResourceUrl(SLIDER_SERVER_XML);
     if (serverXmlUrl != null) {
-
       log.info("Loading {} at {}", SLIDER_SERVER_XML, serverXmlUrl);
-      Configuration serverConf =
-          ConfigHelper.loadFromResource(SLIDER_SERVER_XML);
+      Configuration serverConf = ConfigHelper.loadFromResource(SLIDER_SERVER_XML);
       ConfigHelper.mergeConfigurations(customConf, serverConf,
           SLIDER_SERVER_XML, true);
     }
@@ -505,7 +504,8 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     addService(executorService);
 
     addService(actionQueues);
-    
+    addService(yarnClient = new SliderYarnClientImpl());
+
     //init all child services
     super.serviceInit(conf);
   }
@@ -630,8 +630,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
     Configuration serviceConf = getConfig();
 
-    securityConfiguration = new SecurityConfiguration(
-        serviceConf, instanceDefinition, clustername);
+    securityConfiguration = new SecurityConfiguration(serviceConf, instanceDefinition, clustername);
     // obtain security state
     securityEnabled = securityConfiguration.isSecurityEnabled();
     // set the global security flag for the instance definition
@@ -659,8 +658,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     providerService = factory.createServerProvider();
     // init the provider BUT DO NOT START IT YET
     initAndAddService(providerService);
-    providerRMOperationHandler =
-        new ProviderNotifyingOperationHandler(providerService);
+    providerRMOperationHandler = new ProviderNotifyingOperationHandler(providerService);
     
     // create a slider AM provider
     sliderAMProvider = new SliderAMProviderService();
@@ -682,12 +680,9 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     ApplicationId appid = appAttemptID.getApplicationId();
     log.info("AM for ID {}", appid.getId());
 
-    appInformation.put(StatusKeys.INFO_AM_CONTAINER_ID,
-                       appMasterContainerID.toString());
-    appInformation.put(StatusKeys.INFO_AM_APP_ID,
-                       appid.toString());
-    appInformation.put(StatusKeys.INFO_AM_ATTEMPT_ID,
-                       appAttemptID.toString());
+    appInformation.put(StatusKeys.INFO_AM_CONTAINER_ID, appMasterContainerID.toString());
+    appInformation.put(StatusKeys.INFO_AM_APP_ID, appid.toString());
+    appInformation.put(StatusKeys.INFO_AM_ATTEMPT_ID, appAttemptID.toString());
 
     Map<String, String> envVars;
     List<Container> liveContainers;
@@ -715,7 +710,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
       if (securityEnabled) {
         // fix up the ACLs if they are not set
-        String acls = getConfig().get(KEY_PROTOCOL_ACL);
+        String acls = serviceConf.get(KEY_PROTOCOL_ACL);
         if (acls == null) {
           getConfig().set(KEY_PROTOCOL_ACL, "*");
         }
@@ -731,8 +726,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       appMasterHostname = rpcServiceAddress.getHostName();
       appMasterRpcPort = rpcServiceAddress.getPort();
       appMasterTrackingUrl = null;
-      log.info("AM Server is listening at {}:{}", appMasterHostname,
-               appMasterRpcPort);
+      log.info("AM Server is listening at {}:{}", appMasterHostname, appMasterRpcPort);
       appInformation.put(StatusKeys.INFO_AM_HOSTNAME, appMasterHostname);
       appInformation.set(StatusKeys.INFO_AM_RPC_PORT, appMasterRpcPort);
 
@@ -776,24 +770,27 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       // *****************************************************
       log.info("Connecting to RM at {},address tracking URL={}",
                appMasterRpcPort, appMasterTrackingUrl);
-      amRegistrationData = asyncRMClient
-        .registerApplicationMaster(appMasterHostname,
+      amRegistrationData = asyncRMClient.registerApplicationMaster(appMasterHostname,
                                    appMasterRpcPort,
                                    appMasterTrackingUrl);
-      Resource maxResources =
-        amRegistrationData.getMaximumResourceCapability();
-      containerMaxMemory = maxResources.getMemory();
-      containerMaxCores = maxResources.getVirtualCores();
-      appState.setContainerLimits(maxResources.getMemory(),
-                                  maxResources.getVirtualCores());
+      maximumResourceCapability = amRegistrationData.getMaximumResourceCapability();
+
+      int minMemory = serviceConf.getInt(RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+          DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
+       // validate scheduler vcores allocation setting
+      int minCores = serviceConf.getInt(RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
+          DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+      int maxMemory = maximumResourceCapability.getMemory();
+      int maxCores = maximumResourceCapability.getVirtualCores();
+      appState.setContainerLimits(minMemory,maxMemory, minCores, maxCores );
 
       // build the handler for RM request/release operations; this uses
       // the max value as part of its lookup
-      rmOperationHandler = new AsyncRMOperationHandler(asyncRMClient, maxResources);
+      rmOperationHandler = new AsyncRMOperationHandler(asyncRMClient, maximumResourceCapability);
 
       // set the RM-defined maximum cluster values
-      appInformation.put(ResourceKeys.YARN_CORES, Integer.toString(containerMaxCores));
-      appInformation.put(ResourceKeys.YARN_MEMORY, Integer.toString(containerMaxMemory));
+      appInformation.put(ResourceKeys.YARN_CORES, Integer.toString(maxCores));
+      appInformation.put(ResourceKeys.YARN_MEMORY, Integer.toString(maxMemory));
 
       processAMCredentials(securityConfiguration);
 
@@ -837,15 +834,17 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       Path historyDir = new Path(clusterDirPath, HISTORY_DIR_NAME);
 
       //build the instance
-      appState.buildInstance(instanceDefinition,
-          serviceConf,
-          providerConf,
-          providerRoles,
-          fs.getFileSystem(),
-          historyDir,
-          liveContainers,
-          appInformation,
-          providerService.createContainerReleaseSelector());
+      AppStateBindingInfo binding = new AppStateBindingInfo();
+      binding.instanceDefinition = instanceDefinition;
+      binding.serviceConfig = serviceConf;
+      binding.publishedProviderConf = providerConf;
+      binding.roles = providerRoles;
+      binding.fs = fs.getFileSystem();
+      binding.historyPath = historyDir;
+      binding.liveContainers = liveContainers;
+      binding.applicationInfo = appInformation;
+      binding.releaseSelector = providerService.createContainerReleaseSelector();
+      appState.buildInstance(binding);
 
       providerService.rebuildContainerDetails(liveContainers,
           instanceDefinition.getName(), appState.getRolePriorityMap());

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/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 eadb1dc..1325148 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
@@ -21,7 +21,6 @@ package org.apache.slider.server.appmaster.state;
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.MetricRegistry;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -265,15 +264,17 @@ public class AppState {
 
 
   /**
-   * Record of the max no. of cores allowed in this cluster
+   * limits of container core numbers in this queue
    */
   private int containerMaxCores;
+  private int containerMinCores;
 
   /**
-   * limit container memory
+   * limits of container memory in this queue
    */
   private int containerMaxMemory;
-  
+  private int containerMinMemory;
+
   private RoleHistory roleHistory;
   private Configuration publishedProviderConf;
   private long startTimeThreshold;
@@ -447,27 +448,29 @@ public class AppState {
   }
 
   /**
-   * Set the container limits -the max that can be asked for,
-   * which are used when the "max" values are requested
+   * Set the container limits -the min and max values for
+   * resource requests. All requests must be multiples of the min
+   * values.
+   * @param minMemory min memory MB
    * @param maxMemory maximum memory
+   * @param minCores min v core count
    * @param maxCores maximum cores
    */
-  public void setContainerLimits(int maxMemory, int maxCores) {
+  public void setContainerLimits(int minMemory,int maxMemory, int minCores, int maxCores) {
+    containerMinCores = minCores;
     containerMaxCores = maxCores;
+    containerMinMemory = minMemory;
     containerMaxMemory = maxMemory;
   }
 
-
   public ConfTreeOperations getResourcesSnapshot() {
     return resourcesSnapshot;
   }
 
-
   public ConfTreeOperations getAppConfSnapshot() {
     return appConfSnapshot;
   }
 
-
   public ConfTreeOperations getInternalsSnapshot() {
     return internalsSnapshot;
   }
@@ -488,38 +491,17 @@ public class AppState {
     return unresolvedInstanceDefinition;
   }
 
-  /**
-   * Build up the application state
-   * @param instanceDefinition definition of the applicatin instance
-   * @param appmasterConfig
-   * @param publishedProviderConf any configuration info to be published by a provider
-   * @param providerRoles roles offered by a provider
-   * @param fs filesystem
-   * @param historyDir directory containing history files
-   * @param liveContainers list of live containers supplied on an AM restart
-   * @param applicationInfo app info to retain for web views
-   * @param releaseSelector selector of containers to release
-   */
-  public synchronized void buildInstance(AggregateConf instanceDefinition,
-      Configuration appmasterConfig,
-      Configuration publishedProviderConf,
-      List<ProviderRole> providerRoles,
-      FileSystem fs,
-      Path historyDir,
-      List<Container> liveContainers,
-      Map<String, String> applicationInfo,
-      ContainerReleaseSelector releaseSelector)
-      throws  BadClusterStateException, BadConfigException, IOException {
-    Preconditions.checkArgument(instanceDefinition != null);
-    Preconditions.checkArgument(releaseSelector != null);
+  public synchronized void buildInstance(AppStateBindingInfo binding)
+      throws BadClusterStateException, BadConfigException, IOException {
+    binding.validate();
 
     log.debug("Building application state");
-    this.publishedProviderConf = publishedProviderConf;
-    this.applicationInfo = applicationInfo != null ? applicationInfo
-                                                   : new HashMap<String, String>();
+    publishedProviderConf = binding.publishedProviderConf;
+    applicationInfo = binding.applicationInfo != null ? binding.applicationInfo
+                        : new HashMap<String, String>();
 
     clientProperties = new HashMap<>();
-    containerReleaseSelector = releaseSelector;
+    containerReleaseSelector = binding.releaseSelector;
 
 
     Set<String> confKeys = ConfigHelper.sortedConfigKeys(publishedProviderConf);
@@ -532,15 +514,15 @@ public class AppState {
 
     // set the cluster specification (once its dependency the client properties
     // is out the way
-    setInitialInstanceDefinition(instanceDefinition);
+    setInitialInstanceDefinition(binding.instanceDefinition);
 
     //build the initial role list
-    for (ProviderRole providerRole : providerRoles) {
+    List<ProviderRole> roleList = new ArrayList<>(binding.roles);
+    for (ProviderRole providerRole : roleList) {
       buildRole(providerRole);
     }
 
-    ConfTreeOperations resources =
-        instanceDefinition.getResourceOperations();
+    ConfTreeOperations resources = instanceDefinition.getResourceOperations();
 
     Set<String> roleNames = resources.getComponentNames();
     for (String name : roleNames) {
@@ -551,16 +533,14 @@ public class AppState {
         ProviderRole dynamicRole =
             createDynamicProviderRole(name, resComponent);
         buildRole(dynamicRole);
-        providerRoles.add(dynamicRole);
+        roleList.add(dynamicRole);
       }
     }
     //then pick up the requirements
     buildRoleRequirementsFromResources();
 
-
     //set the livespan
-    MapOperations globalResOpts =
-        instanceDefinition.getResourceOperations().getGlobalOptions();
+    MapOperations globalResOpts = instanceDefinition.getResourceOperations().getGlobalOptions();
     
     startTimeThreshold = globalResOpts.getOptionInt(
         InternalKeys.INTERNAL_CONTAINER_FAILURE_SHORTLIFE,
@@ -576,16 +556,15 @@ public class AppState {
 
 
     // set up the role history
-    roleHistory = new RoleHistory(providerRoles);
+    roleHistory = new RoleHistory(roleList);
     roleHistory.register(metricsAndMonitoring);
-    roleHistory.onStart(fs, historyDir);
+    roleHistory.onStart(binding.fs, binding.historyPath);
 
     //rebuild any live containers
-    rebuildModelFromRestart(liveContainers);
+    rebuildModelFromRestart(binding.liveContainers);
 
     // any am config options to pick up
-    logServerURL = appmasterConfig.get(YarnConfiguration.YARN_LOG_SERVER_URL, "");
-    
+    logServerURL = binding.serviceConfig.get(YarnConfiguration.YARN_LOG_SERVER_URL, "");
     //mark as live
     applicationLive = true;
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java
new file mode 100644
index 0000000..184c8aa
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.providers.ProviderRole;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Binding information for application states; designed to be extensible
+ * so that tests don't have to be massivley reworked when new arguments
+ * are added.
+ */
+public class AppStateBindingInfo {
+  public AggregateConf instanceDefinition;
+  public Configuration serviceConfig = new Configuration();
+  public Configuration publishedProviderConf = new Configuration(false);
+  public List<ProviderRole> roles = new ArrayList<>();
+  public FileSystem fs;
+  public Path historyPath;
+  public List<Container> liveContainers = new ArrayList<>(0);
+  public Map<String, String> applicationInfo = new HashMap<>();
+  public ContainerReleaseSelector releaseSelector = new SimpleReleaseSelector();
+
+  public void validate() throws IllegalArgumentException {
+    Preconditions.checkArgument(instanceDefinition != null, "null instanceDefinition");
+    Preconditions.checkArgument(serviceConfig != null, "null appmasterConfig");
+    Preconditions.checkArgument(publishedProviderConf != null, "null publishedProviderConf");
+    Preconditions.checkArgument(releaseSelector != null, "null releaseSelector");
+    Preconditions.checkArgument(roles != null, "null providerRoles");
+    Preconditions.checkArgument(fs != null, "null fs");
+    Preconditions.checkArgument(historyPath != null, "null historyDir");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
index df3983a..d9a6b34 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
@@ -31,7 +31,6 @@ import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.core.exceptions.BadConfigException;
 import org.apache.slider.providers.ProviderRole;
 import org.apache.slider.server.appmaster.management.BoolMetric;
-import org.apache.slider.server.appmaster.management.LongGauge;
 import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
 import org.apache.slider.server.appmaster.management.Timestamp;
 import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
@@ -52,6 +51,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 
 /**
  * The Role History.
@@ -87,8 +87,8 @@ public class RoleHistory {
   private RoleHistoryWriter historyWriter = new RoleHistoryWriter();
 
   /**
-   * When were the nodes updated in a {@link #onNodesUpdated(List)} call.
-   * If zero: never
+   * When were the nodes updated in a {@link #onNodesUpdated(List)} call?
+   * If zero: never.
    */
   private final Timestamp nodesUpdatedTime = new Timestamp(0);
   private final BoolMetric nodeUpdateReceived = new BoolMetric(false);
@@ -98,9 +98,10 @@ public class RoleHistory {
 
   /**
    * For each role, lists nodes that are available for data-local allocation,
-   ordered by more recently released - To accelerate node selection
+   * ordered by more recently released - to accelerate node selection.
+   * That is, they are "recently used nodes"
    */
-  private Map<Integer, LinkedList<NodeInstance>> availableNodes;
+  private Map<Integer, LinkedList<NodeInstance>> recentNodes;
 
   /**
    * Track the failed nodes. Currently used to make wiser decision of container
@@ -158,8 +159,7 @@ public class RoleHistory {
     throws BadConfigException {
     int index = providerRole.id;
     if (index < 0) {
-      throw new BadConfigException("Provider " + providerRole
-                                               + " id is out of range");
+      throw new BadConfigException("Provider " + providerRole + " id is out of range");
     }
     if (roleStats.get(index) != null) {
       throw new BadConfigException(
@@ -206,7 +206,7 @@ public class RoleHistory {
    * Clear the lists of available nodes
    */
   private synchronized void resetAvailableNodeLists() {
-    availableNodes = new HashMap<>(roleSize);
+    recentNodes = new ConcurrentHashMap<>(roleSize);
   }
 
   /**
@@ -363,7 +363,7 @@ public class RoleHistory {
   public synchronized void insert(Collection<NodeInstance> nodes) {
     nodemap.insert(nodes);
   }
-  
+
   /**
    * Get current time. overrideable for test subclasses
    * @return current time in millis
@@ -435,8 +435,7 @@ public class RoleHistory {
    * @param historyDir path in FS for history
    * @return true if the history was thawed
    */
-  public boolean onStart(FileSystem fs, Path historyDir) throws
-                                                         BadConfigException {
+  public boolean onStart(FileSystem fs, Path historyDir) throws BadConfigException {
     assert filesystem == null;
     filesystem = fs;
     historyPath = historyDir;
@@ -483,7 +482,7 @@ public class RoleHistory {
       }
 
       //start is then completed
-      buildAvailableNodeLists();
+      buildRecentNodeLists();
     } else {
       //fallback to bootstrap procedure
       onBootstrap();
@@ -496,7 +495,7 @@ public class RoleHistory {
    * (After the start), rebuild the availability data structures
    */
   @VisibleForTesting
-  public synchronized void buildAvailableNodeLists() {
+  public synchronized void buildRecentNodeLists() {
     resetAvailableNodeLists();
     // build the list of available nodes
     for (Map.Entry<String, NodeInstance> entry : nodemap.entrySet()) {
@@ -505,13 +504,13 @@ public class RoleHistory {
         NodeEntry nodeEntry = ni.get(i);
         if (nodeEntry != null && nodeEntry.isAvailable()) {
           log.debug("Adding {} for role {}", ni, i);
-          getOrCreateNodesForRoleId(i).add(ni);
+          listRecentNodesForRoleId(i).add(ni);
         }
       }
     }
     // sort the resulting arrays
     for (int i = 0; i < roleSize; i++) {
-      sortAvailableNodeList(i);
+      sortRecentNodeList(i);
     }
   }
 
@@ -521,30 +520,35 @@ public class RoleHistory {
    * @return potentially null list
    */
   @VisibleForTesting
-  public List<NodeInstance> getNodesForRoleId(int id) {
-    return availableNodes.get(id);
+  public List<NodeInstance> getRecentNodesForRoleId(int id) {
+    return recentNodes.get(id);
   }
-  
+
   /**
-   * Get the nodes for an ID -may be null
+   * Get a possibly emtpy list of suggested nodes for a role.
    * @param id role ID
    * @return list
    */
-  private LinkedList<NodeInstance> getOrCreateNodesForRoleId(int id) {
-    LinkedList<NodeInstance> instances = availableNodes.get(id);
+  private LinkedList<NodeInstance> listRecentNodesForRoleId(int id) {
+    LinkedList<NodeInstance> instances = recentNodes.get(id);
     if (instances == null) {
-      instances = new LinkedList<>();
-      availableNodes.put(id, instances);
+      synchronized (this) {
+        // recheck in the synchronized block and recreate
+        if (recentNodes.get(id) == null) {
+          recentNodes.put(id, new LinkedList<NodeInstance>());
+        }
+        instances = recentNodes.get(id);
+      }
     }
     return instances;
   }
-  
+
   /**
-   * Sort an available node list
+   * Sort a the recent node list for a single role
    * @param role role to sort
    */
-  private void sortAvailableNodeList(int role) {
-    List<NodeInstance> nodesForRoleId = getNodesForRoleId(role);
+  private void sortRecentNodeList(int role) {
+    List<NodeInstance> nodesForRoleId = getRecentNodesForRoleId(role);
     if (nodesForRoleId != null) {
       Collections.sort(nodesForRoleId, new NodeInstance.Preferred(role));
     }
@@ -566,7 +570,7 @@ public class RoleHistory {
     NodeInstance nodeInstance = null;
     // Get the list of possible targets.
     // This is a live list: changes here are preserved
-    List<NodeInstance> targets = getNodesForRoleId(roleId);
+    List<NodeInstance> targets = getRecentNodesForRoleId(roleId);
     if (targets == null) {
       // nothing to allocate on
       return null;
@@ -655,7 +659,7 @@ public class RoleHistory {
   public synchronized List<NodeInstance> listActiveNodes(int role) {
     return nodemap.listActiveNodes(role);
   }
-  
+
   /**
    * Get the node entry of a container
    * @param container container to look up
@@ -705,7 +709,7 @@ public class RoleHistory {
    * @return list of containers potentially reordered
    */
   public synchronized List<Container> prepareAllocationList(List<Container> allocatedContainers) {
-    
+
     //partition into requested and unrequested
     List<Container> requested =
       new ArrayList<>(allocatedContainers.size());
@@ -717,7 +721,7 @@ public class RoleHistory {
     requested.addAll(unrequested);
     return requested;
   }
-  
+
   /**
    * A container has been allocated on a node -update the data structures
    * @param container container
@@ -730,7 +734,7 @@ public class RoleHistory {
       int actualCount) {
     int role = ContainerPriority.extractRole(container);
     String hostname = RoleHistoryUtils.hostnameOf(container);
-    List<NodeInstance> nodeInstances = getOrCreateNodesForRoleId(role);
+    List<NodeInstance> nodeInstances = listRecentNodesForRoleId(role);
     ContainerAllocation outcome =
         outstandingRequests.onContainerAllocated(role, hostname, container);
     if (desiredCount <= actualCount) {
@@ -741,7 +745,7 @@ public class RoleHistory {
         //add the list
         log.info("Adding {} hosts for role {}", hosts.size(), role);
         nodeInstances.addAll(hosts);
-        sortAvailableNodeList(role);
+        sortRecentNodeList(role);
       }
     }
     return outcome;
@@ -892,7 +896,7 @@ public class RoleHistory {
 
   /**
    * If the node is marked as available; queue it for assignments.
-   * Unsynced: expects caller to be in a sync block.
+   * Unsynced: requires caller to be in a sync block.
    * @param container completed container
    * @param nodeEntry node
    * @param available available flag
@@ -907,7 +911,7 @@ public class RoleHistory {
       NodeInstance ni = getOrCreateNodeInstance(container);
       int roleId = ContainerPriority.extractRole(container);
       log.debug("Node {} is now available for role id {}", ni, roleId);
-      getOrCreateNodesForRoleId(roleId).addFirst(ni);
+      listRecentNodesForRoleId(roleId).addFirst(ni);
     }
     return available;
   }
@@ -918,8 +922,7 @@ public class RoleHistory {
   public synchronized void dump() {
     for (ProviderRole role : providerRoles) {
       log.info(role.toString());
-      List<NodeInstance> instances =
-        getOrCreateNodesForRoleId(role.id);
+      List<NodeInstance> instances = listRecentNodesForRoleId(role.id);
       log.info("  available: " + instances.size()
                + " " + SliderUtils.joinWithInnerSeparator(" ", instances));
     }
@@ -952,7 +955,7 @@ public class RoleHistory {
    */
   @VisibleForTesting
   public List<NodeInstance> cloneAvailableList(int role) {
-    return new LinkedList<>(getOrCreateNodesForRoleId(role));
+    return new LinkedList<>(listRecentNodesForRoleId(role));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicHistory.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicHistory.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicHistory.groovy
index aa7bb11..c62eb72 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicHistory.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicHistory.groovy
@@ -34,6 +34,7 @@ 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.ContainerRequestOperation
 import org.apache.slider.server.appmaster.state.AppState
+import org.apache.slider.server.appmaster.state.AppStateBindingInfo
 import org.apache.slider.server.appmaster.state.NodeInstance
 import org.apache.slider.server.appmaster.state.RoleInstance
 import org.apache.slider.server.appmaster.state.SimpleReleaseSelector
@@ -47,11 +48,6 @@ import org.junit.Test
 class TestMockAppStateDynamicHistory extends BaseMockAppStateTest
     implements MockRoles {
 
-  @Override
-  String getTestName() {
-    return "TestMockAppStateDynamicHistory"
-  }
-
   /**
    * Small cluster with multiple containers per node,
    * to guarantee many container allocations on each node
@@ -62,26 +58,6 @@ class TestMockAppStateDynamicHistory extends BaseMockAppStateTest
     return new MockYarnEngine(8, 1)
   }
 
-  @Override
-  void initApp() {
-    super.initApp()
-    appState = new MockAppState()
-    appState.setContainerLimits(RM_MAX_RAM, RM_MAX_CORES)
-
-    def instance = factory.newInstanceDefinition(0,0,0)
-
-    appState.buildInstance(
-        instance,
-        new Configuration(),
-        new Configuration(false),
-        factory.ROLES,
-        fs,
-        historyPath,
-        null,
-        null, new SimpleReleaseSelector())
-  }
-
-
   @Test
   public void testDynamicRoleHistory() throws Throwable {
 
@@ -199,7 +175,7 @@ class TestMockAppStateDynamicHistory extends BaseMockAppStateTest
     assert !entry.live
 
 
-    def nodesForRoleId = roleHistory.getNodesForRoleId(role_priority_8)
+    def nodesForRoleId = roleHistory.getRecentNodesForRoleId(role_priority_8)
     assert nodesForRoleId
     
     // make sure new nodes will default to a different host in the engine

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.groovy
index ee4abd6..e35f028 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.groovy
@@ -23,6 +23,7 @@ import groovy.util.logging.Slf4j
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.yarn.api.records.ContainerId
 import org.apache.slider.api.ResourceKeys
+import org.apache.slider.core.conf.AggregateConf
 import org.apache.slider.providers.PlacementPolicy
 import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
 import org.apache.slider.server.appmaster.model.mock.MockAppState
@@ -31,6 +32,7 @@ import org.apache.slider.server.appmaster.model.mock.MockYarnEngine
 import org.apache.slider.server.appmaster.operations.AbstractRMOperation
 import org.apache.slider.server.appmaster.operations.ContainerRequestOperation
 import org.apache.slider.server.appmaster.state.AppState
+import org.apache.slider.server.appmaster.state.AppStateBindingInfo
 import org.apache.slider.server.appmaster.state.ContainerPriority
 import org.apache.slider.server.appmaster.state.RoleHistoryUtils
 import org.apache.slider.server.appmaster.state.RoleInstance
@@ -65,40 +67,27 @@ class TestMockAppStateDynamicRoles extends BaseMockAppStateTest
   }
 
   @Override
-  void initApp() {
-    super.initApp()
-    appState = new MockAppState()
-    appState.setContainerLimits(RM_MAX_RAM, RM_MAX_CORES)
-    def instance = factory.newInstanceDefinition(0,0,0)
-
+  AggregateConf buildInstanceDefinition() {
+    def instance = factory.newInstanceDefinition(0, 0, 0)
     def opts = [
-        (ResourceKeys.COMPONENT_PRIORITY): ROLE4,
+        (ResourceKeys.COMPONENT_PRIORITY) : ROLE4,
         (ResourceKeys.COMPONENT_INSTANCES): "1",
     ]
 
 
-    instance.resourceOperations.components[ROLE4]= opts
+    instance.resourceOperations.components[ROLE4] = opts
 
     def opts5 = [
-        (ResourceKeys.COMPONENT_PRIORITY) : ROLE5,
-        (ResourceKeys.COMPONENT_INSTANCES): "1",
+        (ResourceKeys.COMPONENT_PRIORITY)        : ROLE5,
+        (ResourceKeys.COMPONENT_INSTANCES)       : "1",
         (ResourceKeys.COMPONENT_PLACEMENT_POLICY):
             Integer.toString(PlacementPolicy.STRICT),
-        (ResourceKeys.NODE_FAILURE_THRESHOLD):
+        (ResourceKeys.NODE_FAILURE_THRESHOLD)    :
             Integer.toString(2),
     ]
 
-    instance.resourceOperations.components[ROLE5]= opts5
-
-    appState.buildInstance(
-        instance,
-        new Configuration(),
-        new Configuration(false),
-        factory.ROLES,
-        fs,
-        historyPath,
-        null,
-        null, new SimpleReleaseSelector())
+    instance.resourceOperations.components[ROLE5] = opts5
+    instance
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexDynamicRoles.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexDynamicRoles.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexDynamicRoles.groovy
index 5d880b4..7bc6fe4 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexDynamicRoles.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexDynamicRoles.groovy
@@ -20,21 +20,22 @@ package org.apache.slider.server.appmaster.model.appstate
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 import org.apache.slider.api.ResourceKeys
+import org.apache.slider.core.conf.AggregateConf
 import org.apache.slider.core.conf.ConfTreeOperations
 import org.apache.slider.core.exceptions.BadConfigException
 import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
 import org.apache.slider.server.appmaster.model.mock.MockAppState
 import org.apache.slider.server.appmaster.model.mock.MockRoles
 import org.apache.slider.server.appmaster.model.mock.MockYarnEngine
+import org.apache.slider.server.appmaster.state.AppStateBindingInfo
 import org.apache.slider.server.appmaster.state.MostRecentContainerReleaseSelector
 import org.apache.slider.server.avro.RoleHistoryWriter
 import org.junit.Test
 
 /**
- * Test that if you have >1 role, the right roles are chosen for release.
+ * Test that if you have more than one role, the right roles are chosen for release.
  */
 @CompileStatic
 @Slf4j
@@ -57,32 +58,25 @@ class TestMockAppStateFlexDynamicRoles extends BaseMockAppStateTest
   }
 
   @Override
-  void initApp() {
-    super.initApp()
-    appState = new MockAppState()
-    appState.setContainerLimits(RM_MAX_RAM, RM_MAX_CORES)
+  AppStateBindingInfo buildBindingInfo() {
+    def bindingInfo = super.buildBindingInfo()
+    bindingInfo.releaseSelector = new MostRecentContainerReleaseSelector()
+    bindingInfo
+  }
 
+  @Override
+  AggregateConf buildInstanceDefinition() {
     def instance = factory.newInstanceDefinition(0, 0, 0)
 
     def opts = [
         (ResourceKeys.COMPONENT_INSTANCES): "1",
-        (ResourceKeys.COMPONENT_PRIORITY): "6",
+        (ResourceKeys.COMPONENT_PRIORITY) : "6",
     ]
 
     instance.resourceOperations.components["dynamic-6"] = opts
-
-    
-    appState.buildInstance(instance,
-        new Configuration(),
-        new Configuration(false),
-        factory.ROLES,
-        fs,
-        historyPath,
-        null, null,
-        new MostRecentContainerReleaseSelector())
+    instance
   }
 
-  
   private ConfTreeOperations init() {
     createAndStartNodes();
     def resources = appState.instanceDefinition.resources;
@@ -181,16 +175,10 @@ class TestMockAppStateFlexDynamicRoles extends BaseMockAppStateTest
     def historyWorkDir2 = new File("target/history" + testName + "-0002")
     def historyPath2 = new Path(historyWorkDir2.toURI())
     appState = new MockAppState()
-    appState.setContainerLimits(RM_MAX_RAM, RM_MAX_CORES)
-    appState.buildInstance(
-        factory.newInstanceDefinition(0, 0, 0),
-        new Configuration(),
-        new Configuration(false),
-        factory.ROLES,
-        fs,
-        historyPath2,
-        null, null,
-        new MostRecentContainerReleaseSelector())
+    def binding2 = buildBindingInfo()
+    binding2.instanceDefinition = factory.newInstanceDefinition(0, 0, 0)
+    binding2.historyPath = historyPath2
+    appState.buildInstance(binding2)
     // on this read there won't be the right number of roles
     def loadedRoleHistory = historyWriter.read(fs, history)
     assert 0 == appState.roleHistory.rebuild(loadedRoleHistory)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRebuildOnAMRestart.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRebuildOnAMRestart.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRebuildOnAMRestart.groovy
index c310583..02052c2 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRebuildOnAMRestart.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRebuildOnAMRestart.groovy
@@ -20,14 +20,11 @@ package org.apache.slider.server.appmaster.model.appstate
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.yarn.api.records.Container
 import org.apache.slider.api.StatusKeys
 import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
 import org.apache.slider.server.appmaster.model.mock.MockAppState
 import org.apache.slider.server.appmaster.model.mock.MockRoles
-import org.apache.slider.server.appmaster.operations.AbstractRMOperation
-import org.apache.slider.server.appmaster.state.MostRecentContainerReleaseSelector
 import org.apache.slider.server.appmaster.state.NodeEntry
 import org.apache.slider.server.appmaster.state.NodeInstance
 import org.apache.slider.server.appmaster.state.NodeMap
@@ -35,7 +32,7 @@ import org.apache.slider.server.appmaster.state.RoleInstance
 import org.junit.Test
 
 /**
- * Test that if you have >1 role, the right roles are chosen for release.
+ * Test that app state is rebuilt on a restart
  */
 @CompileStatic
 @Slf4j
@@ -74,16 +71,11 @@ class TestMockAppStateRebuildOnAMRestart extends BaseMockAppStateTest
     appState = new MockAppState()
 
     //and rebuild
-    appState.buildInstance(
-        factory.newInstanceDefinition(r0, r1, r2),
-        new Configuration(),
-        new Configuration(false),
-        factory.ROLES,
-        fs,
-        historyPath,
-        containers,
-        null,
-        new MostRecentContainerReleaseSelector())
+
+    def bindingInfo = buildBindingInfo()
+    bindingInfo.instanceDefinition = factory.newInstanceDefinition(r0, r1, r2)
+    bindingInfo.liveContainers = containers
+    appState.buildInstance(bindingInfo)
 
     assert appState.startedCountainerCount == clusterSize
 
@@ -107,22 +99,18 @@ class TestMockAppStateRebuildOnAMRestart extends BaseMockAppStateTest
       assertNotNull("Null entry in original nodemap for " + hostname, orig)
 
       for (int i = 0; i < ROLE_COUNT; i++) {
-        
-        assert (nodeInstance.getActiveRoleInstances(i) ==
-                orig.getActiveRoleInstances(i))
+        assert (nodeInstance.getActiveRoleInstances(i) == orig.getActiveRoleInstances(i))
         NodeEntry origRE = orig.getOrCreate(i)
         NodeEntry newRE = nodeInstance.getOrCreate(i)
         assert origRE.live == newRE.live
-        assert newRE.starting == 0
+        assert 0 == newRE.starting
       }
     }
-    List<AbstractRMOperation> ops = appState.reviewRequestAndReleaseNodes()
-    assert ops.size() == 0
+    assert 0 == appState.reviewRequestAndReleaseNodes().size()
 
     def status = appState.getClusterStatus()
     // verify the AM restart container count was set
-    String restarted = status.getInfo(
-        StatusKeys.INFO_CONTAINERS_AM_RESTART)
+    String restarted = status.getInfo(StatusKeys.INFO_CONTAINERS_AM_RESTART)
     assert restarted != null;
     //and that the count == 1 master + the region servers
     assert Integer.parseInt(restarted) == containers.size()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.groovy
index 4ba0afd..ad607cf 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.groovy
@@ -25,6 +25,7 @@ import org.apache.slider.api.ResourceKeys
 import org.apache.slider.core.conf.ConfTree
 import org.apache.slider.core.conf.ConfTreeOperations
 import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
+import org.apache.slider.server.appmaster.model.mock.MockAppState
 import org.apache.slider.server.appmaster.model.mock.MockRoles
 import org.apache.slider.server.appmaster.operations.AbstractRMOperation
 import org.apache.slider.server.appmaster.operations.ContainerRequestOperation
@@ -71,7 +72,7 @@ class TestMockContainerResourceAllocations extends BaseMockAppStateTest {
     assert ops.size() == 1
     ContainerRequestOperation operation = (ContainerRequestOperation) ops[0]
     Resource requirements = operation.request.capability
-    assert requirements.memory == RM_MAX_RAM
+    assert requirements.memory == MockAppState.RM_MAX_RAM
     assert requirements.virtualCores == 2
   }
   
@@ -89,7 +90,7 @@ class TestMockContainerResourceAllocations extends BaseMockAppStateTest {
     ContainerRequestOperation operation = (ContainerRequestOperation) ops[0]
     Resource requirements = operation.request.capability
     assert requirements.memory == 512
-    assert requirements.virtualCores == RM_MAX_CORES
+    assert requirements.virtualCores == MockAppState.RM_MAX_CORES
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.groovy
index fa10145..8ab63aa 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.groovy
@@ -68,7 +68,7 @@ class TestRoleHistoryContainerEvents extends BaseMockAppStateTest {
   public void setupRH() {
     roleHistory.onStart(fs, historyPath)
     roleHistory.insert(nodes)
-    roleHistory.buildAvailableNodeLists();
+    roleHistory.buildRecentNodeLists();
     resource = Resource.newInstance(ResourceKeys.DEF_YARN_CORES,
                                     ResourceKeys.DEF_YARN_MEMORY);
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryFindNodesForNewInstances.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryFindNodesForNewInstances.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryFindNodesForNewInstances.groovy
index 79d23e5..c4768ec 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryFindNodesForNewInstances.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryFindNodesForNewInstances.groovy
@@ -63,7 +63,7 @@ class TestRoleHistoryFindNodesForNewInstances extends BaseMockAppStateTest {
   @Before
   public void setupNodeMap() {
     roleHistory.insert(nodes)
-    roleHistory.buildAvailableNodeLists();
+    roleHistory.buildRecentNodeLists();
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
index c81c686..7afcfc1 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
@@ -26,11 +26,9 @@ 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.MockRoles
 import org.apache.slider.server.appmaster.state.NodeEntry
 import org.apache.slider.server.appmaster.state.NodeInstance
 import org.apache.slider.server.appmaster.state.RoleHistory
-import org.apache.slider.server.avro.LoadedRoleHistory
 import org.apache.slider.server.avro.RoleHistoryWriter
 import org.junit.Test
 
@@ -136,7 +134,7 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
     assert rh2.thawedDataTime == savetime
 
     // now start it
-    rh2.buildAvailableNodeLists();
+    rh2.buildRecentNodeLists();
     describe("starting")
     rh2.dump();
     List<NodeInstance> available0 = rh2.cloneAvailableList(0)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
index 9847992..db795d0 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
@@ -67,7 +67,7 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
   @Before
   public void setupNodeMap() {
     roleHistory.insert(nodes)
-    roleHistory.buildAvailableNodeLists();
+    roleHistory.buildRecentNodeLists();
   }
 
   @Test
@@ -141,7 +141,7 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
     recordAsFailed(age4Active1, key, 4)
 
     // trigger a list rebuild
-    roleHistory.buildAvailableNodeLists();
+    roleHistory.buildRecentNodeLists();
 
     assert !roleHistory.cloneAvailableList(key).isEmpty()
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
index 29eefa5..33ea0a0 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
@@ -20,7 +20,6 @@ package org.apache.slider.server.appmaster.model.mock
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.FileSystem as HadoopFS
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.yarn.api.records.Container
@@ -34,21 +33,19 @@ import org.apache.slider.core.conf.AggregateConf
 import org.apache.slider.core.main.LauncherExitCodes
 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.ContainerOutcome
 import org.apache.slider.server.appmaster.state.NodeEntry
 import org.apache.slider.server.appmaster.state.NodeInstance
 import org.apache.slider.server.appmaster.state.RoleInstance
 import org.apache.slider.server.appmaster.state.RoleStatus
-import org.apache.slider.server.appmaster.state.SimpleReleaseSelector
 import org.apache.slider.test.SliderTestBase
 import org.junit.Before
 
 @CompileStatic
 @Slf4j
 abstract class BaseMockAppStateTest extends SliderTestBase implements MockRoles {
-  public static final int RM_MAX_RAM = 4096
-  public static final int RM_MAX_CORES = 64
   MockFactory factory = new MockFactory()
   MockAppState appState
   MockYarnEngine engine
@@ -77,12 +74,14 @@ abstract class BaseMockAppStateTest extends SliderTestBase implements MockRoles
     return new MockYarnEngine(64, 1)
   }
 
+  /**
+   * Initialize the application.
+   * This uses the binding information supplied by {@link #buildBindingInfo()}.
+   */
   @Before
   void initApp(){
 
     String historyDirName = testName;
-
-
     YarnConfiguration conf = SliderUtils.createConfiguration()
     applicationId = new MockApplicationId(id: 1, clusterTimestamp: 0)
     applicationAttemptId = new MockApplicationAttemptId(
@@ -94,29 +93,35 @@ abstract class BaseMockAppStateTest extends SliderTestBase implements MockRoles
     historyPath = new Path(historyWorkDir.toURI())
     fs.delete(historyPath, true)
     appState = new MockAppState()
-    appState.setContainerLimits(RM_MAX_RAM, RM_MAX_CORES)
-    appState.buildInstance(
-        buildInstanceDefinition(),
-        new Configuration(),
-        new Configuration(false),
-        factory.ROLES,
-        fs,
-        historyPath,
-        null, null,
-        new SimpleReleaseSelector())
+    appState.buildInstance(buildBindingInfo())
   }
 
   /**
-   * Override point, define the instance definition
+   * Build the binding info from the default constructor values,
+   * the roles from {@link #factory}, and an instance definition
+   * from {@link #buildInstanceDefinition()}
    * @return
    */
+  AppStateBindingInfo buildBindingInfo() {
+    AppStateBindingInfo binding = new AppStateBindingInfo()
+    binding.instanceDefinition = buildInstanceDefinition();
+    binding.roles = factory.ROLES
+    binding.fs = fs
+    binding.historyPath = historyPath
+    binding
+  }
+
+  /**
+   * Override point, define the instance definition
+   * @return the instance definition
+   */
   public AggregateConf buildInstanceDefinition() {
     factory.newInstanceDefinition(0, 0, 0)
   }
 
   /**
    * Get the test name ... defaults to method name
-   * @return
+   * @return the method name
    */
   String getTestName() {
     methodName.methodName;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockAppState.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockAppState.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockAppState.groovy
index 6e21a38..5565e6b 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockAppState.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockAppState.groovy
@@ -21,12 +21,14 @@ import org.apache.slider.providers.ProviderRole
 import org.apache.slider.server.appmaster.management.MetricsAndMonitoring
 import org.apache.slider.server.appmaster.state.AbstractClusterServices
 import org.apache.slider.server.appmaster.state.AppState
+import org.apache.slider.server.appmaster.state.AppStateBindingInfo
 
 /**
  * Extended app state that makes more things public
  */
 class MockAppState extends AppState {
-
+  public static final int RM_MAX_RAM = 4096
+  public static final int RM_MAX_CORES = 64
   public MockAppState(AbstractClusterServices recordFactory) {
     super(recordFactory, new MetricsAndMonitoring());
   }
@@ -38,6 +40,12 @@ class MockAppState extends AppState {
    */
   public MockAppState() {
     super(new MockClusterServices(), new MetricsAndMonitoring());
+    setContainerLimits(1, RM_MAX_RAM, 1, RM_MAX_CORES)
+  }
+
+  MockAppState(AppStateBindingInfo bindingInfo) {
+    this()
+    buildInstance(bindingInfo)
   }
 
   public Map<String, ProviderRole> getRoleMap() {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
index 4c43168..7237ff4 100644
--- a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
+++ b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
@@ -24,10 +24,10 @@ import com.sun.jersey.api.client.WebResource;
 import com.sun.jersey.api.client.config.ClientConfig;
 import com.sun.jersey.api.client.config.DefaultClientConfig;
 import com.sun.jersey.api.json.JSONConfiguration;
-import junit.framework.Assert;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.slider.common.SliderKeys;
@@ -35,10 +35,12 @@ import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.core.conf.MapOperations;
 import org.apache.slider.core.exceptions.SliderException;
 import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
+import org.apache.slider.server.appmaster.model.mock.MockAppState;
 import org.apache.slider.server.appmaster.model.mock.MockFactory;
 import org.apache.slider.server.appmaster.model.mock.MockProviderService;
 import org.apache.slider.server.appmaster.model.mock.MockClusterServices;
 import org.apache.slider.server.appmaster.state.AppState;
+import org.apache.slider.server.appmaster.state.AppStateBindingInfo;
 import org.apache.slider.server.appmaster.state.ProviderAppState;
 import org.apache.slider.server.appmaster.state.SimpleReleaseSelector;
 import org.apache.slider.server.appmaster.web.WebAppApi;
@@ -71,26 +73,19 @@ public class TestAMAgentWebServices {
 
           public boolean verify(String hostname,
                                 javax.net.ssl.SSLSession sslSession) {
-            if (hostname.equals("localhost")) {
-              return true;
-            }
-            return false;
+            return hostname.equals("localhost");
           }
         });
 
-
   }
 
   protected static final Logger log =
     LoggerFactory.getLogger(TestAMAgentWebServices.class);
   
-  public static final int RM_MAX_RAM = 4096;
-  public static final int RM_MAX_CORES = 64;
   public static final String AGENT_URL =
     "https://localhost:${PORT}/ws/v1/slider/agents/";
   
   static MockFactory factory = new MockFactory();
-  private static Configuration conf = new Configuration();
   private static WebAppApi slider;
 
   private static FileSystem fs;
@@ -117,28 +112,16 @@ public class TestAMAgentWebServices {
     YarnConfiguration conf = SliderUtils.createConfiguration();
     fs = FileSystem.get(new URI("file:///"), conf);
     AppState appState = null;
-    try {
-      fs = FileSystem.get(new URI("file:///"), conf);
-      File
-          historyWorkDir =
-          new File("target/history", "TestAMAgentWebServices");
-      org.apache.hadoop.fs.Path
-          historyPath =
-          new org.apache.hadoop.fs.Path(historyWorkDir.toURI());
-      fs.delete(historyPath, true);
-      appState = new AppState(new MockClusterServices(), new MetricsAndMonitoring());
-      appState.setContainerLimits(RM_MAX_RAM, RM_MAX_CORES);
-      appState.buildInstance(
-          factory.newInstanceDefinition(0, 0, 0),
-          new Configuration(),
-          new Configuration(false),
-          factory.ROLES,
-          fs,
-          historyPath,
-          null, null, new SimpleReleaseSelector());
-    } catch (Exception e) {
-      log.error("Failed to set up app {}", e, e);
-    }
+    File historyWorkDir = new File("target/history", "TestAMAgentWebServices");
+    Path historyPath = new Path(historyWorkDir.toURI());
+    fs.delete(historyPath, true);
+    appState = new MockAppState(new MockClusterServices());
+    AppStateBindingInfo binding = new AppStateBindingInfo();
+    binding.instanceDefinition = factory.newInstanceDefinition(0, 0, 0);
+    binding.roles = MockFactory.ROLES;
+    binding.fs = fs;
+    binding.historyPath = historyPath;
+    appState.buildInstance(binding);
     ProviderAppState providerAppState = new ProviderAppState("undefined",
                                                              appState);
 
@@ -173,7 +156,7 @@ public class TestAMAgentWebServices {
     WebResource webResource = client.resource(base_url + "test/register");
     response = webResource.type(MediaType.APPLICATION_JSON)
         .post(RegistrationResponse.class, createDummyJSONRegister());
-    Assert.assertEquals(RegistrationStatus.OK, response.getResponseStatus());
+    assertEquals(RegistrationStatus.OK, response.getResponseStatus());
   }
 
   protected Client createTestClient() {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f2f2c766/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
index df7e002..70a6bcb 100644
--- a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
+++ b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
@@ -31,7 +31,6 @@ import com.sun.jersey.test.framework.JerseyTest;
 import com.sun.jersey.test.framework.WebAppDescriptor;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.core.conf.AggregateConf;
@@ -39,13 +38,13 @@ import org.apache.slider.core.conf.ConfTree;
 import org.apache.slider.core.exceptions.BadClusterStateException;
 import org.apache.slider.core.exceptions.BadConfigException;
 import org.apache.slider.core.persist.JsonSerDeser;
-import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
+import org.apache.slider.server.appmaster.model.mock.MockAppState;
+import org.apache.slider.server.appmaster.model.mock.MockClusterServices;
 import org.apache.slider.server.appmaster.model.mock.MockFactory;
 import org.apache.slider.server.appmaster.model.mock.MockProviderService;
-import org.apache.slider.server.appmaster.model.mock.MockClusterServices;
 import org.apache.slider.server.appmaster.state.AppState;
+import org.apache.slider.server.appmaster.state.AppStateBindingInfo;
 import org.apache.slider.server.appmaster.state.ProviderAppState;
-import org.apache.slider.server.appmaster.state.SimpleReleaseSelector;
 import org.apache.slider.server.appmaster.web.WebAppApi;
 import org.apache.slider.server.appmaster.web.WebAppApiImpl;
 import org.apache.slider.server.appmaster.web.rest.AMWebServices;
@@ -67,15 +66,11 @@ import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
 
 public class TestAMManagementWebServices extends JerseyTest {
   protected static final Logger log =
       LoggerFactory.getLogger(TestAMManagementWebServices.class);
-  public static final int RM_MAX_RAM = 4096;
-  public static final int RM_MAX_CORES = 64;
-  public static final String EXAMPLES =
-      "/org/apache/slider/core/conf/examples/";
+  public static final String EXAMPLES = "/org/apache/slider/core/conf/examples/";
   static MockFactory factory = new MockFactory();
   private static Configuration conf = new Configuration();
   private static WebAppApi slider;
@@ -116,40 +111,26 @@ public class TestAMManagementWebServices extends JerseyTest {
     }
 
     protected AggregateConf getAggregateConf() {
-      JsonSerDeser<ConfTree> confTreeJsonSerDeser =
-          new JsonSerDeser<ConfTree>(ConfTree.class);
-      ConfTree internal = null;
-      ConfTree app_conf = null;
-      ConfTree resources = null;
       try {
-        internal =
-            confTreeJsonSerDeser.fromResource(
-                EXAMPLES +"internal.json");
-        app_conf =
-            confTreeJsonSerDeser.fromResource(
-                EXAMPLES + "app_configuration.json");
-        resources =
-            confTreeJsonSerDeser.fromResource(
-                EXAMPLES + "resources.json");
+        JsonSerDeser<ConfTree> confTreeJsonSerDeser = new JsonSerDeser<>(ConfTree.class);
+        AggregateConf aggregateConf = new AggregateConf(
+            confTreeJsonSerDeser.fromResource(EXAMPLES + "internal.json"),
+            confTreeJsonSerDeser.fromResource(EXAMPLES + "app_configuration.json"),
+            confTreeJsonSerDeser.fromResource(EXAMPLES + "resources.json"));
+        aggregateConf.setName("test");
+        return aggregateConf;
       } catch (IOException e) {
-        fail(e.getMessage());
+        throw new AssertionError(e.getMessage(), e);
       }
-      AggregateConf aggregateConf = new AggregateConf(
-          resources,
-          app_conf,
-          internal);
-      aggregateConf.setName("test");
-      return aggregateConf;
     }
-
   }
+
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
     injector = createInjector();
-    YarnConfiguration conf = SliderUtils.createConfiguration();
-    fs = FileSystem.get(new URI("file:///"), conf);
+    fs = FileSystem.get(new URI("file:///"), SliderUtils.createConfiguration());
   }
 
   private static Injector createInjector() {
@@ -160,23 +141,17 @@ public class TestAMManagementWebServices extends JerseyTest {
         AppState appState = null;
         try {
           fs = FileSystem.get(new URI("file:///"), conf);
-          File
-              historyWorkDir =
-              new File("target/history", "TestAMManagementWebServices");
-          org.apache.hadoop.fs.Path
-              historyPath =
+          File historyWorkDir = new File("target/history", "TestAMManagementWebServices");
+          org.apache.hadoop.fs.Path historyPath =
               new org.apache.hadoop.fs.Path(historyWorkDir.toURI());
           fs.delete(historyPath, true);
-          appState = new AppState(new MockClusterServices(), new MetricsAndMonitoring());
-          appState.setContainerLimits(RM_MAX_RAM, RM_MAX_CORES);
-          appState.buildInstance(
-              factory.newInstanceDefinition(0, 0, 0),
-              new Configuration(),
-              new Configuration(false),
-              factory.ROLES,
-              fs,
-              historyPath,
-              null, null, new SimpleReleaseSelector());
+          appState = new MockAppState(new MockClusterServices());
+          AppStateBindingInfo binding = new AppStateBindingInfo();
+          binding.instanceDefinition = factory.newInstanceDefinition(0, 0, 0);
+          binding.roles = MockFactory.ROLES;
+          binding.fs = fs;
+          binding.historyPath = historyPath;
+          appState.buildInstance(binding);
         } catch (IOException | BadClusterStateException | URISyntaxException | BadConfigException e) {
           log.error("{}", e, e);
         }


[10/22] incubator-slider git commit: SLIDER-963 Write mock/unit tests for AA placement

Posted by st...@apache.org.
SLIDER-963 Write mock/unit tests for AA placement


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: ee8a9bec05a6808feb06c15ef0de1cf9f96c4a16
Parents: d2ea853
Author: Steve Loughran <st...@apache.org>
Authored: Thu Nov 5 18:48:52 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Nov 5 18:48:52 2015 +0000

----------------------------------------------------------------------
 .../appstate/TestMockAppStateAAPlacement.groovy | 94 ++++++++++++++++++++
 .../model/history/TestRoleHistoryAA.groovy      | 33 +++++++
 2 files changed, 127 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ee8a9bec/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
new file mode 100644
index 0000000..0e9fad0
--- /dev/null
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAAPlacement.groovy
@@ -0,0 +1,94 @@
+/*
+ * 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.hadoop.yarn.api.records.Container
+import org.apache.hadoop.yarn.api.records.NodeId
+import org.apache.hadoop.yarn.client.api.AMRMClient
+import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
+import org.apache.slider.server.appmaster.model.mock.MockRoles
+import org.apache.slider.server.appmaster.operations.AbstractRMOperation
+import org.apache.slider.server.appmaster.operations.CancelSingleRequest
+import org.apache.slider.server.appmaster.operations.ContainerReleaseOperation
+import org.apache.slider.server.appmaster.operations.ContainerRequestOperation
+import org.apache.slider.server.appmaster.state.ContainerAssignment
+import org.apache.slider.server.appmaster.state.RoleHistoryUtils
+import org.apache.slider.server.appmaster.state.RoleInstance
+import org.junit.Test
+
+import static org.apache.slider.server.appmaster.state.ContainerPriority.extractRole
+
+/**
+ * Test Anti-affine placement
+ */
+@CompileStatic
+@Slf4j
+class TestMockAppStateAAPlacement extends BaseMockAppStateTest
+    implements MockRoles {
+
+  @Test
+  public void testAllocateAA() throws Throwable {
+
+    def aaRole = role2Status
+
+    aaRole.desired = 2
+
+    List<AbstractRMOperation> ops = appState.reviewRequestAndReleaseNodes()
+    assert 1 == ops.size()
+    ContainerRequestOperation operation = (ContainerRequestOperation) ops[0]
+    AMRMClient.ContainerRequest request = operation.request
+    assert request.relaxLocality
+    assert request.nodes == null
+    assert request.racks == null
+    assert request.capability
+
+    Container allocated = engine.allocateContainer(request)
+
+    // node is allocated wherever
+
+    def firstAllocation = allocated.nodeId
+
+    // notify the container ane expect
+    List<ContainerAssignment> assignments = [];
+    List<AbstractRMOperation> releaseOperations = []
+    appState.onContainersAllocated([allocated], assignments, releaseOperations)
+
+    // verify the release matches the allocation
+    assert releaseOperations.size() == 1
+    CancelSingleRequest cancelOp = releaseOperations[0] as CancelSingleRequest;
+    assert cancelOp.request.capability.equals(allocated.resource)
+    // now the assignment
+    assert assignments.size() == 1
+
+    // we also expect a new allocation request to have been issued
+    //
+
+    ContainerAssignment assigned = assignments[0]
+    Container container = assigned.container
+    RoleInstance ri = roleInstance(assigned)
+    //tell the app it arrived
+    appState.containerStartSubmitted(container, ri);
+    assert appState.onNodeManagerContainerStarted(container.id)
+    ops = appState.reviewRequestAndReleaseNodes()
+    assert ops.size() == 0
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ee8a9bec/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryAA.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryAA.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryAA.groovy
new file mode 100644
index 0000000..36b9d66
--- /dev/null
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryAA.groovy
@@ -0,0 +1,33 @@
+/*
+ * 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.history
+
+import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
+import org.junit.Test
+
+/**
+ * Test anti-affine
+ */
+class TestRoleHistoryAA extends BaseMockAppStateTest {
+
+  @Test
+  public void test() throws Throwable {
+    
+  }
+}


[02/22] incubator-slider git commit: SLIDER-82 setting up node listings into AppState binding

Posted by st...@apache.org.
SLIDER-82 setting up node listings into AppState binding


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: 1312bc33c6b4739c6be96ecc95fa3f3391baf73f
Parents: 9cda83b
Author: Steve Loughran <st...@apache.org>
Authored: Wed Nov 4 18:11:28 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Nov 5 13:19:03 2015 +0000

----------------------------------------------------------------------
 .../slider/client/SliderYarnClientImpl.java     | 18 +++++++++--
 .../server/appmaster/SliderAppMaster.java       | 34 ++++++++++++--------
 .../appmaster/state/AppStateBindingInfo.java    |  3 ++
 .../slider/agent/rest/TestStandaloneREST.groovy |  4 +--
 .../appmaster/model/mock/MockYarnCluster.groovy | 11 +++----
 5 files changed, 44 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1312bc33/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java b/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
index 42759fd..803ccd6 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
@@ -20,6 +20,7 @@ package org.apache.slider.client;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
@@ -29,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
@@ -42,6 +44,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.net.BindException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
@@ -64,6 +67,17 @@ public class SliderYarnClientImpl extends YarnClientImpl {
    */
   public static final String KILL_ALL = "all";
 
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    String addr = conf.get(YarnConfiguration.RM_ADDRESS);
+    if (addr.startsWith("0.0.0.0")) {
+      // address isn't known; fail fast
+      throw new BindException("Invalid " + YarnConfiguration.RM_ADDRESS + " value:" + addr
+          + " - see https://wiki.apache.org/hadoop/UnsetHostnameOrPort");
+    }
+    super.serviceInit(conf);
+  }
+
   /**
    * Get the RM Client RPC interface
    * @return an RPC interface valid after initialization and authentication
@@ -107,7 +121,6 @@ public class SliderYarnClientImpl extends YarnClientImpl {
     return results;
   }
 
-
   /**
    * find all instances of a specific app -if there is more than one in the
    * YARN cluster,
@@ -141,8 +154,7 @@ public class SliderYarnClientImpl extends YarnClientImpl {
   public boolean isApplicationLive(ApplicationReport app) {
     Preconditions.checkArgument(app != null, "Null app report");
 
-    return app.getYarnApplicationState().ordinal() <=
-           YarnApplicationState.RUNNING.ordinal();
+    return app.getYarnApplicationState().ordinal() <= YarnApplicationState.RUNNING.ordinal();
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1312bc33/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 b552290..e6a5bd5 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
@@ -57,9 +57,9 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
 import org.apache.hadoop.yarn.client.api.async.NMClientAsync;
@@ -68,7 +68,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import static org.apache.hadoop.yarn.conf.YarnConfiguration.*;
 import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.registry.client.api.RegistryOperations;
 import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
@@ -173,6 +172,7 @@ import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URL;
@@ -215,8 +215,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
    */
   protected static final Logger LOG_YARN = log;
 
-  public static final String SERVICE_CLASSNAME_SHORT =
-      "SliderAppMaster";
+  public static final String SERVICE_CLASSNAME_SHORT = "SliderAppMaster";
   public static final String SERVICE_CLASSNAME =
       "org.apache.slider.server.appmaster." + SERVICE_CLASSNAME_SHORT;
 
@@ -495,8 +494,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     metrics.registerAll(new GarbageCollectorMetricSet());
 
 */
-    contentCache = ApplicationResouceContentCacheFactory.createContentCache(
-        stateForProviders);
+    contentCache = ApplicationResouceContentCacheFactory.createContentCache(stateForProviders);
 
     executorService = new WorkflowExecutorService<>("AmExecutor",
         Executors.newFixedThreadPool(2,
@@ -504,6 +502,14 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     addService(executorService);
 
     addService(actionQueues);
+    // set up the YARN client. This may require patching in the RM client-API address if it
+    // is (somehow) unset server-side.
+    String clientRMaddr = conf.get(YarnConfiguration.RM_ADDRESS);
+    if (clientRMaddr.startsWith("0.0.0.0")) {
+      // address isn't known; fail fast
+      throw new BindException("Invalid " + YarnConfiguration.RM_ADDRESS + " value:" + addr
+          + " - see https://wiki.apache.org/hadoop/UnsetHostnameOrPort");
+    }
     addService(yarnClient = new SliderYarnClientImpl());
 
     //init all child services
@@ -564,8 +570,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
     //dump the system properties if in debug mode
     if (log.isDebugEnabled()) {
-      log.debug("System properties:\n" +
-                SliderUtils.propertiesToString(System.getProperties()));
+      log.debug("System properties:\n" + SliderUtils.propertiesToString(System.getProperties()));
     }
 
     //choose the action
@@ -634,8 +639,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     // obtain security state
     securityEnabled = securityConfiguration.isSecurityEnabled();
     // set the global security flag for the instance definition
-    instanceDefinition.getAppConfOperations().set(
-        KEY_SECURITY_ENABLED, securityEnabled);
+    instanceDefinition.getAppConfOperations().set(KEY_SECURITY_ENABLED, securityEnabled);
 
     // triggers resolution and snapshotting for agent
     appState.setInitialInstanceDefinition(instanceDefinition);
@@ -653,8 +657,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       InternalKeys.INTERNAL_PROVIDER_NAME);
     log.info("Cluster provider type is {}", providerType);
     SliderProviderFactory factory =
-      SliderProviderFactory.createSliderProviderFactory(
-          providerType);
+      SliderProviderFactory.createSliderProviderFactory(providerType);
     providerService = factory.createServerProvider();
     // init the provider BUT DO NOT START IT YET
     initAndAddService(providerService);
@@ -673,8 +676,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
      * turned into an (incompete) container
      */
     appMasterContainerID = ConverterUtils.toContainerId(
-      SliderUtils.mandatoryEnvVariable(
-          ApplicationConstants.Environment.CONTAINER_ID.name()));
+      SliderUtils.mandatoryEnvVariable(ApplicationConstants.Environment.CONTAINER_ID.name()));
     appAttemptID = appMasterContainerID.getApplicationAttemptId();
 
     ApplicationId appid = appAttemptID.getApplicationId();
@@ -687,6 +689,9 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     Map<String, String> envVars;
     List<Container> liveContainers;
 
+    List<NodeReport> nodeReports = yarnClient.getNodeReports(NodeState.RUNNING);
+    log.info("Yarn node report count: {}", nodeReports.size());
+
     /*
      * It is critical this section is synchronized, to stop async AM events
      * arriving while registering a restarting AM.
@@ -844,6 +849,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       binding.liveContainers = liveContainers;
       binding.applicationInfo = appInformation;
       binding.releaseSelector = providerService.createContainerReleaseSelector();
+      binding.nodeReports = nodeReports;
       appState.buildInstance(binding);
 
       providerService.rebuildContainerDetails(liveContainers,

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1312bc33/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java
index 184c8aa..a2a0b60 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.slider.core.conf.AggregateConf;
 import org.apache.slider.providers.ProviderRole;
 
@@ -46,6 +47,8 @@ public class AppStateBindingInfo {
   public List<Container> liveContainers = new ArrayList<>(0);
   public Map<String, String> applicationInfo = new HashMap<>();
   public ContainerReleaseSelector releaseSelector = new SimpleReleaseSelector();
+  /** node reports off the RM. If null, the app state needs to be given a node update later */
+  public List<NodeReport> nodeReports = new ArrayList<>(0);
 
   public void validate() throws IllegalArgumentException {
     Preconditions.checkArgument(instanceDefinition != null, "null instanceDefinition");

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1312bc33/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
index 97b3009..29fa51a 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/rest/TestStandaloneREST.groovy
@@ -41,7 +41,6 @@ import static org.apache.slider.server.appmaster.web.rest.RestPaths.*
 @Slf4j
 class TestStandaloneREST extends AgentMiniClusterTestBase {
 
-
   @Test
   public void testStandaloneREST() throws Throwable {
 
@@ -65,8 +64,7 @@ class TestStandaloneREST extends AgentMiniClusterTestBase {
     ApplicationReport report = waitForClusterLive(client)
     def proxyAM = report.trackingUrl
     def directAM = report.originalTrackingUrl
-    
-    
+
     // set up url config to match
     initHttpTestSupport(launcher.configuration)
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/1312bc33/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnCluster.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnCluster.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnCluster.groovy
index 6056e3a..99a9213 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnCluster.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnCluster.groovy
@@ -147,14 +147,15 @@ public class MockYarnCluster {
   
 /**
  * Model cluster nodes on the simpler "slot" model than the YARN-era
- * resource allocation model. Why? Makes it easier to implement.
+ * resource allocation model. Why? Easier to implement scheduling.
+ * Of course, if someone does want to implement the full process...
  *
- * When a cluster is offline, 
  */
   public static class MockYarnClusterNode {
 
     public final int nodeIndex
     public final String hostname;
+    public List<String> labels = []
     public final MockNodeId nodeId;
     public final MockYarnClusterContainer[] containers;
     private boolean offline;
@@ -230,8 +231,6 @@ public class MockYarnCluster {
       }
       return result
     }
-    
-    
 
     /**
      * Release a container
@@ -291,8 +290,8 @@ public class MockYarnCluster {
     return (hostIndex << 8) | containerIndex & 0xff;
   }
 
-  public static final int extractHost(int cid) {
-    return (cid >>> 8);
+  public static final int extractHost(long cid) {
+    return (cid >>> 8) & 0xffff;
   }
 
   public static final int extractContainer(int cid) {


[07/22] incubator-slider git commit: Merge branch 'develop' into feature/SLIDER-82-pass-3

Posted by st...@apache.org.
Merge branch 'develop' into feature/SLIDER-82-pass-3


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: 55d50fe87e2deb00aad9775f809f4f694d82cf92
Parents: 5887dde fed5d03
Author: Steve Loughran <st...@apache.org>
Authored: Thu Nov 5 15:24:32 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Nov 5 15:24:32 2015 +0000

----------------------------------------------------------------------
 .../src/main/java/org/apache/slider/client/SliderClient.java    | 4 ++--
 .../apache/slider/providers/agent/TestAgentClientProvider2.java | 5 +++++
 2 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/55d50fe8/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------


[05/22] incubator-slider git commit: SLIDER-961 clean up SliderClient code -methods moved to CommandLineBuilder should really have gone into JavaCommandLineBuilder

Posted by st...@apache.org.
SLIDER-961 clean up SliderClient code -methods moved to CommandLineBuilder should really have gone into JavaCommandLineBuilder


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: f1bad85f3f26622d5339abea1f3de353be06ce82
Parents: 3b9b221
Author: Steve Loughran <st...@apache.org>
Authored: Thu Nov 5 13:24:14 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Nov 5 14:04:05 2015 +0000

----------------------------------------------------------------------
 .../slider/core/launch/CommandLineBuilder.java  | 55 ------------------
 .../core/launch/JavaCommandLineBuilder.java     | 59 +++++++++++++++++++-
 2 files changed, 58 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f1bad85f/slider-core/src/main/java/org/apache/slider/core/launch/CommandLineBuilder.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/CommandLineBuilder.java b/slider-core/src/main/java/org/apache/slider/core/launch/CommandLineBuilder.java
index 57b8965..dbaa981 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/CommandLineBuilder.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/CommandLineBuilder.java
@@ -19,11 +19,8 @@
 package org.apache.slider.core.launch;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
-import org.apache.slider.common.params.Arguments;
 import org.apache.slider.common.tools.SliderUtils;
-import org.apache.slider.core.exceptions.BadConfigException;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -104,56 +101,4 @@ public class CommandLineBuilder {
     return argumentList;
   }
 
-  public boolean addConfOption(Configuration conf, String key) {
-    String val = conf.get(key);
-    return defineIfSet(key, val);
-  }
-
-  public String addConfOptionToCLI(Configuration conf,
-      String key,
-      String defVal) {
-    String val = conf.get(key, defVal);
-    define(key, val);
-    return val;
-  }
-
-  /**
-   * Add a <code>-D key=val</code> command to the CLI
-   * @param key key
-   * @param val value
-   */
-  public void define(String key, String val) {
-    Preconditions.checkArgument(key != null, "null key");
-    Preconditions.checkArgument(val != null, "null value");
-    add(Arguments.ARG_DEFINE, key + "=" + val);
-  }
-
-  /**
-   * Add a <code>-D key=val</code> command to the CLI if <code>val</code>
-   * is not null
-   * @param key key
-   * @param val value
-   */
-  public boolean defineIfSet(String key, String val) {
-    Preconditions.checkArgument(key != null, "null key");
-    if (val != null) {
-      define(key, val);
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  /**
-   * Add a mandatory config option
-   * @param conf configuration
-   * @param key key
-   * @throws BadConfigException if the key is missing
-   */
-  public void addMandatoryConfOption(Configuration conf,
-      String key) throws BadConfigException {
-    if (!addConfOption(conf, key)) {
-      throw new BadConfigException("Missing configuration option: " + key);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f1bad85f/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java b/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
index 0b3fa10..9197e5d 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
@@ -20,11 +20,15 @@ package org.apache.slider.core.launch;
 
 
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadConfigException;
 
 /**
- * Command line builder purely for the Java CLI
+ * Command line builder purely for the Java CLI.
+ * Some of the <code>define</code> methods are designed to work with Hadoop tool and
+ * Slider launcher applications.
  */
 public class JavaCommandLineBuilder extends CommandLineBuilder {
 
@@ -80,4 +84,57 @@ public class JavaCommandLineBuilder extends CommandLineBuilder {
     sysprop("java.awt.headless", "true");
     return this;
   }
+
+  public boolean addConfOption(Configuration conf, String key) {
+    String val = conf.get(key);
+    return defineIfSet(key, val);
+  }
+
+  public String addConfOptionToCLI(Configuration conf,
+      String key,
+      String defVal) {
+    String val = conf.get(key, defVal);
+    define(key, val);
+    return val;
+  }
+
+  /**
+   * Add a <code>-D key=val</code> command to the CLI. This is very Hadoop API
+   * @param key key
+   * @param val value
+   */
+  public void define(String key, String val) {
+    Preconditions.checkArgument(key != null, "null key");
+    Preconditions.checkArgument(val != null, "null value");
+    add("-D", key + "=" + val);
+  }
+
+  /**
+   * Add a <code>-D key=val</code> command to the CLI if <code>val</code>
+   * is not null
+   * @param key key
+   * @param val value
+   */
+  public boolean defineIfSet(String key, String val) {
+    Preconditions.checkArgument(key != null, "null key");
+    if (val != null) {
+      define(key, val);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Add a mandatory config option
+   * @param conf configuration
+   * @param key key
+   * @throws BadConfigException if the key is missing
+   */
+  public void addMandatoryConfOption(Configuration conf,
+      String key) throws BadConfigException {
+    if (!addConfOption(conf, key)) {
+      throw new BadConfigException("Missing configuration option: " + key);
+    }
+  }
 }


[19/22] incubator-slider git commit: SLIDER-966 RoleHistory now creates the resource requests, issued via the factory. To be honest, I'm not 100% sure I like the current design; I may need to do another iteration

Posted by st...@apache.org.
SLIDER-966 RoleHistory now creates the resource requests, issued via the factory. To be honest, I'm not 100% sure I like the current design; I may need to do another iteration


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: aa46b473e11e9ec154ee12cf5d5113d7b974f99c
Parents: ac98d82
Author: Steve Loughran <st...@apache.org>
Authored: Fri Nov 6 21:19:51 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Nov 6 21:19:51 2015 +0000

----------------------------------------------------------------------
 .../appmaster/ProtobufClusterServices.java      |   6 +
 .../state/AbstractClusterServices.java          |   6 +
 .../slider/server/appmaster/state/AppState.java | 126 +++++++++++--------
 .../server/appmaster/state/RoleHistory.java     |  23 ++--
 .../appstate/TestMockAppStateAAPlacement.groovy |  29 +++--
 .../TestRoleHistoryContainerEvents.groovy       |  62 ++++-----
 .../TestRoleHistoryRequestTracking.groovy       |  32 ++---
 .../model/mock/MockClusterServices.groovy       |   5 +
 .../appmaster/model/mock/MockFactory.groovy     |   4 +-
 .../appmaster/model/mock/MockRoleHistory.groovy |   3 +-
 10 files changed, 182 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/aa46b473/slider-core/src/main/java/org/apache/slider/server/appmaster/ProtobufClusterServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/ProtobufClusterServices.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/ProtobufClusterServices.java
index 50b5dad..5d52441 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/ProtobufClusterServices.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/ProtobufClusterServices.java
@@ -20,6 +20,7 @@ package org.apache.slider.server.appmaster;
 
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.slider.server.appmaster.state.AbstractClusterServices;
 
 public class ProtobufClusterServices extends AbstractClusterServices {
@@ -27,4 +28,9 @@ public class ProtobufClusterServices extends AbstractClusterServices {
   public Resource newResource() {
     return Records.newRecord(Resource.class);
   }
+
+  @Override
+  public Resource newResource(int memory, int cores) {
+    return Resources.createResource(memory, cores);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/aa46b473/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AbstractClusterServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AbstractClusterServices.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AbstractClusterServices.java
index 27e25f9..eba8c38 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AbstractClusterServices.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AbstractClusterServices.java
@@ -19,6 +19,7 @@
 package org.apache.slider.server.appmaster.state;
 
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 
 /**
  * Cluster services offered by the YARN infrastructure.
@@ -30,4 +31,9 @@ public abstract class AbstractClusterServices {
    */
   public abstract Resource newResource();
 
+  public abstract Resource newResource(int memory, int cores);
+
+  public Resource normalize(Resource resource, Resource minR, Resource maxR) {
+    return new DefaultResourceCalculator().normalize(resource, minR, maxR);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/aa46b473/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 f6fe474..1e23bef 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
@@ -289,6 +289,8 @@ public class AppState {
    * Selector of containers to release; application wide.
    */
   private ContainerReleaseSelector containerReleaseSelector;
+  private Resource minResource;
+  private Resource maxResource;
 
   /**
    * Create an instance
@@ -309,7 +311,7 @@ public class AppState {
     register(MetricsConstants.CONTAINERS_START_FAILED, startFailedContainerCount);
   }
 
-  private void register(String name, Counter counter) {
+  private void register(String name, Metric counter) {
     this.metricsAndMonitoring.getMetrics().register(
         MetricRegistry.name(AppState.class, name), counter);
   }
@@ -462,6 +464,8 @@ public class AppState {
     containerMaxCores = maxCores;
     containerMinMemory = minMemory;
     containerMaxMemory = maxMemory;
+    minResource = recordFactory.newResource(containerMinMemory, containerMinCores);
+    maxResource = recordFactory.newResource(containerMaxMemory, containerMaxCores);
   }
 
   public ConfTreeOperations getResourcesSnapshot() {
@@ -556,7 +560,7 @@ public class AppState {
 
 
     // set up the role history
-    roleHistory = new RoleHistory(roleStatusMap.values());
+    roleHistory = new RoleHistory(roleStatusMap.values(), recordFactory);
     roleHistory.register(metricsAndMonitoring);
     roleHistory.onStart(binding.fs, binding.historyPath);
     // trigger first node update
@@ -762,6 +766,9 @@ public class AppState {
         newRoles.add(dynamicRole);
       }
     }
+    // and fill in all those roles with their requirements
+    buildRoleResourceRequirements();
+
     return newRoles;
   }
 
@@ -812,6 +819,17 @@ public class AppState {
   }
 
   /**
+   * Build up the requirements of every resource
+   */
+  private void buildRoleResourceRequirements() {
+    roleStatusMap.values();
+    for (RoleStatus role : roleStatusMap.values()) {
+      role.setResourceRequirements(
+          buildResourceRequirements(role, recordFactory.newResource()));
+    }
+  }
+
+  /**
    * build up the special master node, which lives
    * in the live node set but has a lifecycle bonded to the AM
    * @param containerId the AM master
@@ -913,6 +931,12 @@ public class AppState {
   }
 
 
+  /**
+   * Look up a role in the map
+   * @param name role name
+   * @return the instance
+   * @throws YarnRuntimeException if not found
+   */
   public RoleStatus lookupRoleStatus(String name) throws YarnRuntimeException {
     ProviderRole providerRole = roles.get(name);
     if (providerRole == null) {
@@ -928,7 +952,7 @@ public class AppState {
    */
   public synchronized List<RoleInstance> cloneOwnedContainerList() {
     Collection<RoleInstance> values = ownedContainers.values();
-    return new ArrayList<RoleInstance>(values);
+    return new ArrayList<>(values);
   }
 
   /**
@@ -1027,7 +1051,6 @@ public class AppState {
     }
   }
 
-
   public synchronized List<RoleInstance> getLiveInstancesByContainerIDs(
     Collection<String> containerIDs) {
     //first, a hashmap of those containerIDs is built up
@@ -1081,7 +1104,6 @@ public class AppState {
     return nodes;
   }
 
-
   /**
    * Build an instance map.
    * @return the map of Role name to list of role instances
@@ -1167,38 +1189,18 @@ public class AppState {
 
 
   /**
-   * Set up the resource requirements with all that this role needs, 
-   * then create the container request itself.
-   * @param role role to ask an instance of
-   * @param capability a resource to set up
-   * @return the request for a new container
-   */
-  public AMRMClient.ContainerRequest buildContainerResourceAndRequest(
-        RoleStatus role,
-        Resource capability) {
-    buildResourceRequirements(role, capability);
-    String labelExpression = role.getLabelExpression();
-    //get the role history to select a suitable node, if available
-    AMRMClient.ContainerRequest containerRequest =
-      createContainerRequest(role, capability);
-    return  containerRequest;
-  }
-
-  /**
    * Create a container request.
-   * Update internal state, such as the role request count
-   * This is where role history information will be used for placement decisions -
-   * @param labelExpression label expression to satisfy
+   * Update internal state, such as the role request count. 
+   * Anti-Affine: the {@link RoleStatus#outstandingAArequest} is set here.
+   * This is where role history information will be used for placement decisions.
    * @param role role
-   * @param resource requirements
    * @return the container request to submit
    */
-  private AMRMClient.ContainerRequest createContainerRequest(RoleStatus role,
-      Resource resource) {
-    AMRMClient.ContainerRequest request;
-    request = roleHistory.requestNode(role, resource);
+  private AMRMClient.ContainerRequest createContainerRequest(RoleStatus role) {
     incrementRequestCount(role);
-    return request;
+    OutstandingRequest request = roleHistory.requestContainerForRole(role);
+    role.setOutstandingAArequest(request);
+    return request.getIssuedRequest();
   }
 
   /**
@@ -1266,9 +1268,10 @@ public class AppState {
    * cluster specification, including substituing max allowed values
    * if the specification asked for it.
    * @param role role
-   * @param capability capability to set up
+   * @param capability capability to set up. A new one may be created
+   * during normalization
    */
-  public void buildResourceRequirements(RoleStatus role, Resource capability) {
+  public Resource buildResourceRequirements(RoleStatus role, Resource capability) {
     // Set up resource requirements from role values
     String name = role.getName();
     ConfTreeOperations resources = getResourcesSnapshot();
@@ -1283,6 +1286,7 @@ public class AppState {
                                      DEF_YARN_MEMORY,
                                      containerMaxMemory);
     capability.setMemory(ram);
+    return recordFactory.normalize(capability,minResource, maxResource);
   }
 
   /**
@@ -1748,7 +1752,7 @@ public class AppState {
   public synchronized List<AbstractRMOperation> reviewRequestAndReleaseNodes()
       throws SliderInternalStateException, TriggerClusterTeardownException {
     log.debug("in reviewRequestAndReleaseNodes()");
-    List<AbstractRMOperation> allOperations = new ArrayList<AbstractRMOperation>();
+    List<AbstractRMOperation> allOperations = new ArrayList<>();
     for (RoleStatus roleStatus : getRoleStatusMap().values()) {
       if (!roleStatus.isExcludeFromFlexing()) {
         List<AbstractRMOperation> operations = reviewOneRole(roleStatus);
@@ -1853,6 +1857,7 @@ public class AppState {
     long delta;
     long expected;
     String name = role.getName();
+    boolean isAA = role.isAntiAffinePlacement();
     synchronized (role) {
       delta = role.getDelta();
       expected = role.getDesired();
@@ -1871,38 +1876,43 @@ public class AppState {
     }
 
     if (delta > 0) {
-      log.info("{}: Asking for {} more nodes(s) for a total of {} ", name,
-               delta, expected);
       // more workers needed than we have -ask for more
-      for (int i = 0; i < delta; i++) {
-        Resource capability = recordFactory.newResource();
-        AMRMClient.ContainerRequest containerAsk =
-          buildContainerResourceAndRequest(role, capability);
-        log.info("Container ask is {} and label = {}", containerAsk,
-            containerAsk.getNodeLabelExpression());
-        int askMemory = containerAsk.getCapability().getMemory();
-        if (askMemory > this.containerMaxMemory) {
-          log.warn("Memory requested: {} > max of {}", askMemory, containerMaxMemory);
+      log.info("{}: Asking for {} more nodes(s) for a total of {} ", name,
+          delta, expected);
+
+      // TODO: AA RH to help here by only allowing one request for an AA
+
+      if (isAA) {
+        // build one only if there is none outstanding
+        if (role.getPendingAntiAffineRequests() == 0) {
+          log.info("Starting an anti-affine request sequence");
+          role.incPendingAntiAffineRequests(delta);
+          addContainerRequest(operations, createContainerRequest(role));
+        } else {
+          log.info("Adding {} more anti-affine requests", delta);
+          role.incPendingAntiAffineRequests(delta);
+        }
+      } else {
+
+        for (int i = 0; i < delta; i++) {
+          //get the role history to select a suitable node, if available
+          addContainerRequest(operations, createContainerRequest(role));
         }
-        operations.add(new ContainerRequestOperation(containerAsk));
       }
     } else if (delta < 0) {
       log.info("{}: Asking for {} fewer node(s) for a total of {}", name,
                -delta,
                expected);
       // reduce the number expected (i.e. subtract the delta)
-
-      // then pick some containers to kill
       long excess = -delta;
 
-      // how many requests are outstanding
+      // how many requests are outstanding?
       long outstandingRequests = role.getRequested();
       if (outstandingRequests > 0) {
         // outstanding requests.
         int toCancel = (int)Math.min(outstandingRequests, excess);
 
         // Delegate to Role History
-
         List<AbstractRMOperation> cancellations = roleHistory.cancelRequestsForRole(role, toCancel);
         log.info("Found {} outstanding requests to cancel", cancellations.size());
         operations.addAll(cancellations);
@@ -1922,7 +1932,6 @@ public class AppState {
         }
       }
 
-
       // after the cancellation there may be no excess
       if (excess > 0) {
 
@@ -1936,7 +1945,7 @@ public class AppState {
           log.info("No containers for component {}", roleId);
         }
 
-        // cut all release-in-progress nodes
+        // filter out all release-in-progress nodes
         ListIterator<RoleInstance> li = containersToRelease.listIterator();
         while (li.hasNext()) {
           RoleInstance next = li.next();
@@ -1979,6 +1988,17 @@ public class AppState {
     return operations;
   }
 
+  private void addContainerRequest(List<AbstractRMOperation> operations,
+      AMRMClient.ContainerRequest containerAsk) {
+    log.info("Container ask is {} and label = {}", containerAsk,
+        containerAsk.getNodeLabelExpression());
+    int askMemory = containerAsk.getCapability().getMemory();
+    if (askMemory > this.containerMaxMemory) {
+      log.warn("Memory requested: {} > max of {}", askMemory, containerMaxMemory);
+    }
+    operations.add(new ContainerRequestOperation(containerAsk));
+  }
+
   /**
    * Releases a container based on container id
    * @param containerId

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/aa46b473/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
index c93c7f5..f8271a6 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
@@ -73,6 +73,8 @@ public class RoleHistory {
   private final List<ProviderRole> providerRoles;
   /** the roles in here are shared with App State */
   private final Map<Integer, RoleStatus> roleStatusMap = new HashMap<>();
+  private final AbstractClusterServices recordFactory;
+
   private long startTime;
 
   /** Time when saved */
@@ -115,9 +117,11 @@ public class RoleHistory {
   /**
    * Instantiate
    * @param roles initial role list
+   * @param recordFactory yarn record factory
    * @throws BadConfigException
    */
-  public RoleHistory(Collection<RoleStatus> roles) throws BadConfigException {
+  public RoleHistory(Collection<RoleStatus> roles, AbstractClusterServices recordFactory) throws BadConfigException {
+    this.recordFactory = recordFactory;
     roleSize = roles.size();
     providerRoles = new ArrayList<>(roleSize);
     for (RoleStatus role : roles) {
@@ -604,24 +608,27 @@ public class RoleHistory {
    *
    * @param node node to target or null for "any"
    * @param role role to request
-   * @return the container priority
+   * @return the request
    */
-  public synchronized AMRMClient.ContainerRequest requestInstanceOnNode(
+  public synchronized OutstandingRequest requestInstanceOnNode(
       NodeInstance node, RoleStatus role, Resource resource) {
     OutstandingRequest outstanding = outstandingRequests.newRequest(node, role.getKey());
-    return outstanding.buildContainerRequest(resource, role, now());
+    outstanding.buildContainerRequest(resource, role, now());
+    return outstanding;
   }
 
   /**
    * Find a node for a role and request an instance on that (or a location-less
    * instance)
    * @param role role status
-   * @param resource resource capabilities
    * @return a request ready to go
    */
-  public synchronized AMRMClient.ContainerRequest requestNode(RoleStatus role,
-                                                              Resource resource) {
+  public synchronized OutstandingRequest requestContainerForRole(RoleStatus role) {
+
+    Resource resource = recordFactory.newResource();
+    role.copyResourceRequirements(resource);
     NodeInstance node = findNodeForNewInstance(role);
+    // TODO AA -what if there are no suitable nodes?
     return requestInstanceOnNode(node, role, resource);
   }
 
@@ -992,6 +999,8 @@ public class RoleHistory {
     // ask for some placed nodes
     requests.addAll(outstandingRequests.extractPlacedRequestsForRole(roleId, remaining));
 
+    // TODO AA: clear anything here?
+
     // build cancellations
     for (OutstandingRequest request : requests) {
       results.add(request.createCancelOperation());

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/aa46b473/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 157870a..42772c5 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
@@ -22,6 +22,8 @@ import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.yarn.api.records.Container
 import org.apache.hadoop.yarn.client.api.AMRMClient
+import org.apache.slider.api.ResourceKeys
+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
@@ -43,16 +45,29 @@ import org.junit.Test
 class TestMockAppStateAAPlacement extends BaseMockAppStateTest
     implements MockRoles {
 
-  static private final ProviderRole aaRole = MockFactory.PROVIDER_ROLE2
-  private static final int roleId = aaRole.id
-/*
+  /**
+   * 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)
+
   @Override
   AppStateBindingInfo buildBindingInfo() {
     def bindingInfo = super.buildBindingInfo()
-    // only have the AA role, to avoid complications/confusion
-    bindingInfo.roles = [aaRole]
+    bindingInfo.roles = [
+        MockFactory.PROVIDER_ROLE0,
+        MockFactory.PROVIDER_ROLE1,
+        AAROLE,
+    ]
     bindingInfo
-  }*/
+  }
+
+  private static final int roleId = AAROLE.id
 
   /**
    * Get the single request of a list of operations; includes the check for the size
@@ -74,7 +89,7 @@ class TestMockAppStateAAPlacement extends BaseMockAppStateTest
   @Test
   public void testAllocateAANoLabel() throws Throwable {
 
-    def aaRole = lookupRole(aaRole.name)
+    def aaRole = lookupRole(AAROLE.name)
 
     // want two instances, so there will be two iterations
     aaRole.desired = 2

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/aa46b473/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.groovy
index 5609682..d9cfddb 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.groovy
@@ -35,7 +35,6 @@ import org.apache.slider.server.appmaster.model.mock.MockFactory
 import org.apache.slider.server.appmaster.model.mock.MockNodeId
 import org.apache.slider.server.appmaster.model.mock.MockRoleHistory
 import org.apache.slider.server.appmaster.state.*
-import org.junit.Before
 import org.junit.Test
 
 /**
@@ -61,13 +60,18 @@ class TestRoleHistoryContainerEvents extends BaseMockAppStateTest {
   String roleName = "test"
 
   List<NodeInstance> nodes = [age2Active2, age2Active0, age4Active1, age1Active4, age3Active0]
-  RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES)
+  RoleHistory roleHistory
 
   Resource resource
 
-  @Before
-  public void setupRH() {
-    roleHistory.onStart(fs, historyPath)
+  AMRMClient.ContainerRequest requestContainer(RoleStatus roleStatus) {
+    roleHistory.requestContainerForRole(roleStatus).issuedRequest
+  }
+
+  @Override
+  void setup() {
+    super.setup()
+    roleHistory = appState.roleHistory
     roleHistory.insert(nodes)
     roleHistory.buildRecentNodeLists();
     resource = Resource.newInstance(ResourceKeys.DEF_YARN_CORES,
@@ -77,11 +81,10 @@ class TestRoleHistoryContainerEvents extends BaseMockAppStateTest {
   @Test
   public void testFindAndCreate() throws Throwable {
     int role = 0
-    ProviderRole provRole = new ProviderRole(roleName, role)
-    RoleStatus roleStatus = new RoleStatus(provRole)
+    RoleStatus roleStatus = appState.lookupRoleStatus(role)
 
     AMRMClient.ContainerRequest request =
-        roleHistory.requestNode(roleStatus, resource);
+        requestContainer(roleStatus);
 
     List<String> nodes = request.getNodes()
     assert nodes != null
@@ -114,14 +117,13 @@ class TestRoleHistoryContainerEvents extends BaseMockAppStateTest {
   @Test
   public void testCreateAndRelease() throws Throwable {
     int role = 1
-    ProviderRole provRole = new ProviderRole(roleName, role)
-    RoleStatus roleStatus = new RoleStatus(provRole)
+    RoleStatus roleStatus = appState.lookupRoleStatus(role)
 
     //verify it is empty
     assert roleHistory.listActiveNodes(role).empty
 
     AMRMClient.ContainerRequest request =
-        roleHistory.requestNode(roleStatus, resource);
+        requestContainer(roleStatus);
 
     assert request.nodes == null
 
@@ -170,7 +172,7 @@ class TestRoleHistoryContainerEvents extends BaseMockAppStateTest {
 
     // ask for a container and expect to get the recently released one
     AMRMClient.ContainerRequest request2 =
-        roleHistory.requestNode(roleStatus, resource);
+        requestContainer(roleStatus);
 
     List<String> nodes2 = request2.nodes
     assert nodes2 != null
@@ -206,11 +208,10 @@ class TestRoleHistoryContainerEvents extends BaseMockAppStateTest {
   @Test
   public void testStartFailed() throws Throwable {
     int role = 0
-    ProviderRole provRole = new ProviderRole(roleName, role)
-    RoleStatus roleStatus = new RoleStatus(provRole)
+    RoleStatus roleStatus = appState.lookupRoleStatus(role)
 
     AMRMClient.ContainerRequest request =
-        roleHistory.requestNode(roleStatus, resource);
+        requestContainer(roleStatus);
 
     String hostname = request.getNodes()[0]
     assert hostname == age3Active0.hostname
@@ -240,10 +241,10 @@ class TestRoleHistoryContainerEvents extends BaseMockAppStateTest {
   @Test
   public void testStartFailedWithoutWarning() throws Throwable {
     int role = 0
-    ProviderRole provRole = new ProviderRole(roleName, role)
-    RoleStatus roleStatus = new RoleStatus(provRole)
+    RoleStatus roleStatus = appState.lookupRoleStatus(role)
+
     AMRMClient.ContainerRequest request =
-        roleHistory.requestNode(roleStatus, resource);
+        requestContainer(roleStatus);
 
     String hostname = request.getNodes()[0]
     assert hostname == age3Active0.hostname
@@ -271,10 +272,10 @@ class TestRoleHistoryContainerEvents extends BaseMockAppStateTest {
     describe("fail a container without declaring it as starting")
 
     int role = 0
-    ProviderRole provRole = new ProviderRole(roleName, role)
-    RoleStatus roleStatus = new RoleStatus(provRole)
+    RoleStatus roleStatus = appState.lookupRoleStatus(role)
+
     AMRMClient.ContainerRequest request =
-        roleHistory.requestNode(roleStatus, resource);
+        requestContainer(roleStatus);
 
     String hostname = request.getNodes()[0]
     assert hostname == age3Active0.hostname
@@ -310,10 +311,10 @@ class TestRoleHistoryContainerEvents extends BaseMockAppStateTest {
   public void testContainerFailedWithoutWarning() throws Throwable {
     describe( "fail a container without declaring it as starting")
     int role = 0
-    ProviderRole provRole = new ProviderRole(roleName, role)
-    RoleStatus roleStatus = new RoleStatus(provRole)
+    RoleStatus roleStatus = appState.lookupRoleStatus(role)
+
     AMRMClient.ContainerRequest request =
-        roleHistory.requestNode(roleStatus, resource);
+        requestContainer(roleStatus);
 
     String hostname = request.getNodes()[0]
     assert hostname == age3Active0.hostname
@@ -343,10 +344,10 @@ class TestRoleHistoryContainerEvents extends BaseMockAppStateTest {
   public void testAllocationListPrep() throws Throwable {
     describe("test prepareAllocationList")
     int role = 0
-    ProviderRole provRole = new ProviderRole(roleName, role)
-    RoleStatus roleStatus = new RoleStatus(provRole)
+    RoleStatus roleStatus = appState.lookupRoleStatus(role)
+
     AMRMClient.ContainerRequest request =
-        roleHistory.requestNode(roleStatus, resource);
+        requestContainer(roleStatus);
 
     String hostname = request.getNodes()[0]
     assert hostname == age3Active0.hostname
@@ -375,9 +376,10 @@ class TestRoleHistoryContainerEvents extends BaseMockAppStateTest {
     describe("fail a node")
 
     int role = 0
-    ProviderRole provRole = new ProviderRole(roleName, role)
-    RoleStatus roleStatus = new RoleStatus(provRole)
-    AMRMClient.ContainerRequest request = roleHistory.requestNode(roleStatus, resource);
+    RoleStatus roleStatus = appState.lookupRoleStatus(role)
+
+    AMRMClient.ContainerRequest request =
+        roleHistory.requestContainerForRole(roleStatus).issuedRequest;
 
     String hostname = request.getNodes()[0]
     assert age3Active0.hostname == hostname

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/aa46b473/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
index 693ea9f..14ac32a 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
@@ -63,6 +63,10 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
   ProviderRole provRole = new ProviderRole(roleName, 0)
   RoleStatus roleStatus = new RoleStatus(provRole)
 
+  AMRMClient.ContainerRequest requestContainer(RoleStatus roleStatus) {
+    roleHistory.requestContainerForRole(roleStatus).issuedRequest
+  }
+
   @Override
   String getTestName() {
     return "TestRoleHistoryAvailableList"
@@ -72,6 +76,7 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
   public void setupNodeMap() {
     roleHistory.insert(nodes)
     roleHistory.buildRecentNodeLists();
+    roleStatus.setResourceRequirements(Resource.newInstance(1, 1))
   }
 
   @Test
@@ -107,8 +112,7 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
     // which is translated to a no-location request
     AMRMClient.ContainerRequest req = roleHistory.requestInstanceOnNode(ni,
         roleStatus,
-        resource
-    )
+        resource).issuedRequest
 
     assertNull(req.nodes)
 
@@ -121,7 +125,7 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
     // looking for a node should now find one
     ni = roleHistory.findNodeForNewInstance(roleStatus)
     assert ni == age3Active0
-    req = roleHistory.requestInstanceOnNode(ni, roleStatus, resource)
+    req = roleHistory.requestInstanceOnNode(ni, roleStatus, resource).issuedRequest
     assert 1 == req.nodes.size()
   }
 
@@ -162,7 +166,7 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
 
   @Test
   public void testFindAndRequestNode() throws Throwable {
-    AMRMClient.ContainerRequest req = roleHistory.requestNode(roleStatus, resource)
+    AMRMClient.ContainerRequest req = requestContainer(roleStatus)
 
     assert age3Active0.hostname == req.nodes[0]
     List<NodeInstance> a2 = roleHistory.cloneRecentNodeList(0)
@@ -171,7 +175,7 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
 
   @Test
   public void testRequestedNodeIntoReqList() throws Throwable {
-    AMRMClient.ContainerRequest req = roleHistory.requestNode(roleStatus, resource)
+    requestContainer(roleStatus)
     List<OutstandingRequest> requests = roleHistory.listPlacedRequests()
     assert requests.size() == 1
     assert age3Active0.hostname == requests[0].hostname
@@ -179,7 +183,7 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
 
   @Test
   public void testCompletedRequestDropsNode() throws Throwable {
-    AMRMClient.ContainerRequest req = roleHistory.requestNode(roleStatus, resource)
+    AMRMClient.ContainerRequest req = requestContainer(roleStatus)
     List<OutstandingRequest> requests = roleHistory.listPlacedRequests()
     assert requests.size() == 1
     String hostname = requests[0].hostname
@@ -208,8 +212,8 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
 
   @Test
   public void testTwoRequests() throws Throwable {
-    AMRMClient.ContainerRequest req = roleHistory.requestNode(roleStatus, resource)
-    AMRMClient.ContainerRequest req2 = roleHistory.requestNode(roleStatus, resource)
+    AMRMClient.ContainerRequest req = requestContainer(roleStatus)
+    AMRMClient.ContainerRequest req2 = requestContainer(roleStatus)
     List<OutstandingRequest> requests = roleHistory.listPlacedRequests()
     assert requests.size() == 2
     MockContainer container = factory.newContainer(req, req.nodes[0])
@@ -222,9 +226,9 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
 
   @Test
   public void testThreeRequestsOneUnsatisified() throws Throwable {
-    AMRMClient.ContainerRequest req = roleHistory.requestNode(roleStatus, resource)
-    AMRMClient.ContainerRequest req2 = roleHistory.requestNode(roleStatus, resource)
-    AMRMClient.ContainerRequest req3 = roleHistory.requestNode(roleStatus, resource)
+    AMRMClient.ContainerRequest req = requestContainer(roleStatus)
+    AMRMClient.ContainerRequest req2 = requestContainer(roleStatus)
+    AMRMClient.ContainerRequest req3 = requestContainer(roleStatus)
     List<OutstandingRequest> requests = roleHistory.listPlacedRequests()
     assert requests.size() == 2
     MockContainer container = factory.newContainer(req, req.nodes[0])
@@ -251,9 +255,9 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
 
   @Test
   public void testThreeRequests() throws Throwable {
-    AMRMClient.ContainerRequest req = roleHistory.requestNode(roleStatus, resource)
-    AMRMClient.ContainerRequest req2 = roleHistory.requestNode(roleStatus, resource)
-    AMRMClient.ContainerRequest req3 = roleHistory.requestNode(roleStatus, resource)
+    AMRMClient.ContainerRequest req = requestContainer(roleStatus)
+    AMRMClient.ContainerRequest req2 = requestContainer(roleStatus)
+    AMRMClient.ContainerRequest req3 = requestContainer(roleStatus)
     assertOutstandingPlacedRequests(2)
     assert req3.nodes == null
     MockContainer container = factory.newContainer(req, req.nodes[0])

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/aa46b473/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockClusterServices.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockClusterServices.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockClusterServices.groovy
index d27a6bb..dfecc94 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockClusterServices.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockClusterServices.groovy
@@ -27,4 +27,9 @@ class MockClusterServices extends AbstractClusterServices {
   Resource newResource() {
     return new MockResource()
   }
+
+  @Override
+  Resource newResource(int memory, int cores) {
+    return new MockResource(memory, cores)
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/aa46b473/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 0a4a93e..bbd64f1 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
@@ -59,11 +59,11 @@ class MockFactory implements MockRoles {
       2,
       1,
       ResourceKeys.DEF_YARN_LABEL_EXPRESSION)
-  // role 2: longer delay and anti-affinity
+  // role 2: longer delay
   public static final ProviderRole PROVIDER_ROLE2 = new ProviderRole(
       MockRoles.ROLE2,
       2,
-      PlacementPolicy.ANTI_AFFINITY_REQUIRED,
+      PlacementPolicy.NO_DATA_LOCALITY,
       2,
       2,
       ResourceKeys.DEF_YARN_LABEL_EXPRESSION)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/aa46b473/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockRoleHistory.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockRoleHistory.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockRoleHistory.groovy
index 0a68afb..4553e22 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockRoleHistory.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockRoleHistory.groovy
@@ -35,7 +35,8 @@ class MockRoleHistory extends RoleHistory {
    * @throws BadConfigException configuration problem with the role list
    */
   MockRoleHistory(List<ProviderRole> providerRoles) throws BadConfigException {
-    super(providerRoles.collect { new RoleStatus(it) })
+    super(providerRoles.collect { new RoleStatus(it) },
+      new MockClusterServices())
   }
 
 }


[09/22] incubator-slider git commit: SLIDER-82 generation of onContainerAllocated() cancel requests moved into RoleHistory; this will allow it to also generate new allocations at the same time.

Posted by st...@apache.org.
SLIDER-82 generation of  onContainerAllocated() cancel requests moved into RoleHistory; this will allow it to also generate new allocations at the same time.


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: d2ea8537a74b7b0ba09836d50d55d54dab2dc683
Parents: a3591eb
Author: Steve Loughran <st...@apache.org>
Authored: Thu Nov 5 18:48:41 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Nov 5 18:48:41 2015 +0000

----------------------------------------------------------------------
 .../slider/server/appmaster/state/AppState.java | 26 ++++++++------------
 .../appmaster/state/ContainerAllocation.java    | 14 +++++++----
 .../appmaster/state/OutstandingRequest.java     |  2 +-
 .../state/OutstandingRequestTracker.java        | 22 ++++++++++++++---
 .../server/appmaster/state/RoleHistory.java     |  5 +++-
 ...tRoleHistoryOutstandingRequestTracker.groovy | 13 ++++++++--
 .../model/mock/BaseMockAppStateTest.groovy      |  2 --
 .../appmaster/model/mock/MockFactory.groovy     |  2 +-
 8 files changed, 54 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2ea8537/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 9e29af2..e47ef34 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
@@ -2087,13 +2087,13 @@ public class AppState {
    * a list of operations to perform
    * @param allocatedContainers the containers allocated
    * @param assignments the assignments of roles to containers
-   * @param releaseOperations any release operations
+   * @param operations any allocation or release operations
    */
   public synchronized void onContainersAllocated(List<Container> allocatedContainers,
                                     List<ContainerAssignment> assignments,
-                                    List<AbstractRMOperation> releaseOperations) {
+                                    List<AbstractRMOperation> operations) {
     assignments.clear();
-    releaseOperations.clear();
+    operations.clear();
     List<Container> ordered = roleHistory.prepareAllocationList(allocatedContainers);
     log.debug("onContainersAllocated(): Total containers allocated = {}", ordered.size());
     for (Container container : ordered) {
@@ -2118,23 +2118,14 @@ public class AppState {
           roleHistory.onContainerAllocated(container, desired, allocated);
       final ContainerAllocationOutcome outcome = allocation.outcome;
 
-      // cancel an allocation request which granted this, so as to avoid repeated
-      // requests
-      if (allocation.origin != null && allocation.origin.getIssuedRequest() != null) {
-        releaseOperations.add(allocation.origin.createCancelOperation());
-      } else {
-        // there's a request, but no idea what to cancel.
-        // rather than try to recover from it inelegantly, (and cause more confusion),
-        // log the event, but otherwise continue
-        log.warn("Unexpected allocation of container "
-            + SliderUtils.containerToString(container));
-      }
+      // add all requests to the operations list
+      operations.addAll(allocation.operations);
 
       //look for condition where we get more back than we asked
       if (allocated > desired) {
         log.info("Discarding surplus {} container {} on {}", roleName,  cid,
             containerHostInfo);
-        releaseOperations.add(new ContainerReleaseOperation(cid));
+        operations.add(new ContainerReleaseOperation(cid));
         //register as a surplus node
         surplusNodes.add(cid);
         surplusContainers.inc();
@@ -2156,7 +2147,10 @@ public class AppState {
 
         assignments.add(new ContainerAssignment(container, role, outcome));
         //add to the history
-        roleHistory.onContainerAssigned(container);
+        AbstractRMOperation request = roleHistory.onContainerAssigned(container);
+        if (request != null) {
+          operations.add(request);
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2ea8537/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocation.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocation.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocation.java
index 306ffb2..6bfe8ab 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocation.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocation.java
@@ -18,6 +18,11 @@
 
 package org.apache.slider.server.appmaster.state;
 
+import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
+
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * This is just a tuple of the outcome of a container allocation
  */
@@ -35,11 +40,10 @@ public class ContainerAllocation {
    */
   public OutstandingRequest origin;
 
-  public ContainerAllocation(ContainerAllocationOutcome outcome,
-      OutstandingRequest origin) {
-    this.outcome = outcome;
-    this.origin = origin;
-  }
+  /**
+   * A possibly empty list of requests to add to the follow-up actions
+   */
+  public List<AbstractRMOperation> operations = new ArrayList<>(0);
 
   public ContainerAllocation() {
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2ea8537/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java
index 602b48e..85bd259 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java
@@ -375,7 +375,7 @@ public final class OutstandingRequest {
    * and in mock tests
    *
    */
-  public  void validate() throws InvalidContainerRequestException {
+  public void validate() throws InvalidContainerRequestException {
     Preconditions.checkNotNull(issuedRequest, "request has not yet been built up");
     AMRMClient.ContainerRequest containerRequest = issuedRequest;
     String exp = containerRequest.getNodeLabelExpression();

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2ea8537/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java
index a8787f0..bf34d43 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java
@@ -115,8 +115,14 @@ public class OutstandingRequestTracker {
   }
 
   /**
-   * Notification that a container has been allocated -drop it
-   * from the {@link #placedRequests} structure.
+   * Notification that a container has been allocated
+   *
+   * <ol>
+   *   <li>drop it from the {@link #placedRequests} structure.</li>
+   *   <li>generate the cancellation request</li>
+   *   <li>for AA placement, any actions needed</li>
+   * </ol>
+   *
    * @param role role index
    * @param hostname hostname
    * @return the allocation outcome
@@ -129,8 +135,7 @@ public class OutstandingRequestTracker {
         containerDetails);
     ContainerAllocation allocation = new ContainerAllocation();
     ContainerAllocationOutcome outcome;
-    OutstandingRequest request =
-        placedRequests.remove(new OutstandingRequest(role, hostname));
+    OutstandingRequest request = placedRequests.remove(new OutstandingRequest(role, hostname));
     if (request != null) {
       //satisfied request
       log.debug("Found placed request for container: {}", request);
@@ -154,6 +159,15 @@ public class OutstandingRequestTracker {
         outcome = ContainerAllocationOutcome.Unallocated;
       }
     }
+    if (request != null && request.getIssuedRequest() != null) {
+      allocation.operations.add(request.createCancelOperation());
+    } else {
+      // there's a request, but no idea what to cancel.
+      // rather than try to recover from it inelegantly, (and cause more confusion),
+      // log the event, but otherwise continue
+      log.warn("Unexpected allocation of container " + SliderUtils.containerToString(container));
+    }
+
     allocation.origin = request;
     allocation.outcome = outcome;
     return allocation;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2ea8537/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
index 6880b69..a0aa3bc 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
@@ -749,6 +749,8 @@ public class RoleHistory {
         sortRecentNodeList(role);
       }
     }
+    // AA placement: now request a new node
+
     return outcome;
   }
 
@@ -756,9 +758,10 @@ public class RoleHistory {
    * A container has been assigned to a role instance on a node -update the data structures
    * @param container container
    */
-  public void onContainerAssigned(Container container) {
+  public AbstractRMOperation onContainerAssigned(Container container) {
     NodeEntry nodeEntry = getOrCreateNodeEntry(container);
     nodeEntry.onStarting();
+    return null;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2ea8537/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
index 33cacd7..56b2c31 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
@@ -55,7 +55,11 @@ class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
     tracker.newRequest(host1, 0)
     tracker.newRequest(host2, 0)
     tracker.newRequest(host1, 1)
-    assert tracker.onContainerAllocated(1, "host1", null).outcome == ContainerAllocationOutcome.Placed
+
+    def allocation = tracker.onContainerAllocated(1, "host1", null)
+    assert allocation.outcome == ContainerAllocationOutcome.Placed
+    assert allocation.operations[0] instanceof CancelSingleRequest
+
     assert !tracker.lookupPlacedRequest(1, "host1")
     assert tracker.lookupPlacedRequest(0, "host1")
   }
@@ -83,8 +87,11 @@ class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
     resource.virtualCores=1
     resource.memory = 48;
     c1.setResource(resource)
-    ContainerAllocationOutcome outcome = tracker.onContainerAllocated(0, "host1", c1).outcome
+
+    def allocation = tracker.onContainerAllocated(0, "host1", c1)
+    ContainerAllocationOutcome outcome = allocation.outcome
     assert outcome == ContainerAllocationOutcome.Unallocated
+    assert allocation.operations.empty
     assert tracker.listOpenRequests().size() == 1
   }
 
@@ -115,6 +122,8 @@ class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
 
     def allocation = tracker.onContainerAllocated(0, nodeId.host, c1)
     assert tracker.listOpenRequests().size() == 0
+    assert allocation.operations[0] instanceof CancelSingleRequest
+
     assert allocation.outcome == ContainerAllocationOutcome.Open
     assert allocation.origin.is(req1)
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2ea8537/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
index babce9b..a065518 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
@@ -152,7 +152,6 @@ abstract class BaseMockAppStateTest extends SliderTestBase implements MockRoles
     return ri
   }
 
-
   public NodeInstance nodeInstance(long age, int live0, int live1=0, int live2=0) {
     NodeInstance ni = new NodeInstance("age${age}-[${live0},${live1},$live2]",
                                        MockFactory.ROLE_COUNT)
@@ -205,7 +204,6 @@ abstract class BaseMockAppStateTest extends SliderTestBase implements MockRoles
     return createStartAndStopNodes([])
   }
 
-
   /**
    * Create, Start and stop nodes
    * @param completionResults List filled in with the status on all completed nodes

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d2ea8537/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 6071ef0..25fdd8b 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
@@ -57,7 +57,7 @@ class MockFactory implements MockRoles {
       PlacementPolicy.STRICT,
       2,
       1)
-  // role 2: longer delay
+  // role 2: longer delay and anti-affinity
   public static final ProviderRole PROVIDER_ROLE2 = new ProviderRole(
       MockRoles.ROLE2,
       2,


[21/22] incubator-slider git commit: SLIDER-966 fix a regression from a test that was faling now ROLE2 wasn't AA

Posted by st...@apache.org.
SLIDER-966 fix a regression from a test that was faling now ROLE2 wasn't AA


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: ee0c8daa43a5bdf82a364436305712d287e5916d
Parents: b54eb4a
Author: Steve Loughran <st...@apache.org>
Authored: Fri Nov 6 23:26:33 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Nov 6 23:26:33 2015 +0000

----------------------------------------------------------------------
 .../model/history/TestRoleHistoryOutstandingRequestTracker.groovy   | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ee0c8daa/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
index 745d40f..1c99c04 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
@@ -170,7 +170,6 @@ class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
     // process skips entries which are in the list but have not been issued.
     // ...which can be a race condition between request issuance & escalation.
     // (not one observed outside test authoring, but retained for completeness)
-    assert role2Status.placementPolicy == PlacementPolicy.ANTI_AFFINITY_REQUIRED
     def (res2, outstanding2) = newRequest(role2Status)
 
     // simulate some time escalation of role 1 MUST now be triggered


[15/22] incubator-slider git commit: SLIDER-964 AggregateConfResource exports appconf under internal/

Posted by st...@apache.org.
SLIDER-964 AggregateConfResource exports appconf under internal/


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: 0fce42fb7a6de4c4e9139e439fd304efd4b595df
Parents: ca43d1b
Author: Steve Loughran <st...@apache.org>
Authored: Fri Nov 6 14:08:47 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Nov 6 14:08:47 2015 +0000

----------------------------------------------------------------------
 .../resources/AggregateConfResource.java        |  2 +-
 .../management/TestAMManagementWebServices.java | 20 +++++++++++++++++---
 .../core/conf/examples/app_configuration.json   |  4 +++-
 .../core/conf/examples/internal-resolved.json   |  2 +-
 .../slider/core/conf/examples/internal.json     |  2 +-
 .../slider/core/conf/examples/resources.json    |  2 +-
 6 files changed, 24 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0fce42fb/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java
index ebffd1a..794daf9 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java
@@ -38,7 +38,7 @@ public class AggregateConfResource {
   public AggregateConfResource(AggregateConf conf, UriBuilder uriBuilder) {
     if (uriBuilder != null) {
       this.href = uriBuilder.build().toASCIIString();
-      resources = ResourceFactory.createConfTreeResource(conf.getAppConf(),
+      resources = ResourceFactory.createConfTreeResource(conf.getResources(),
                    uriBuilder.clone().path("configurations").path("resources"));
       internal = ResourceFactory.createConfTreeResource(conf.getInternal(),
                    uriBuilder.clone().path("configurations").path("internal"));

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0fce42fb/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
index 44eb692..49ad71a 100644
--- a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
+++ b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
@@ -66,6 +66,7 @@ import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 public class TestAMManagementWebServices extends JerseyTest {
   protected static final Logger log =
@@ -215,9 +216,19 @@ public class TestAMManagementWebServices extends JerseyTest {
     assertEquals("wrong href",
                  "http://localhost:9998/slideram/ws/v1/slider/mgmt/app/configurations/internal",
                  json.getHref());
-    assertEquals("wrong description",
-        "Internal configuration DO NOT EDIT",
-        json.getMetadata().get("description"));
+
+    assertDescriptionContains("org/apache/slider/core/conf/examples/internal.json", json);
+  }
+
+  private void assertDescriptionContains(String expected, ConfTreeResource json) {
+
+    Map<String, Object> metadata = json.getMetadata();
+    assertNotNull("No metadata", metadata);
+    Object actual = metadata.get("description");
+    assertNotNull("No description", actual);
+
+    assertTrue(String.format("Did not find \"%s\" in \"%s\"", expected, actual),
+        actual.toString().contains(expected));
   }
 
   @Test
@@ -239,6 +250,7 @@ public class TestAMManagementWebServices extends JerseyTest {
     assertNotNull("no components", components);
     assertEquals("incorrect number of components", 2, components.size());
     assertNotNull("wrong component", components.get("worker"));
+    assertDescriptionContains("org/apache/slider/core/conf/examples/resources.json", json);
   }
 
   @Test
@@ -259,5 +271,7 @@ public class TestAMManagementWebServices extends JerseyTest {
     assertNotNull("no components", components);
     assertEquals("incorrect number of components", 2, components.size());
     assertNotNull("wrong component", components.get("worker"));
+    assertDescriptionContains("org/apache/slider/core/conf/examples/app_configuration.json", json);
+
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0fce42fb/slider-core/src/test/resources/org/apache/slider/core/conf/examples/app_configuration.json
----------------------------------------------------------------------
diff --git a/slider-core/src/test/resources/org/apache/slider/core/conf/examples/app_configuration.json b/slider-core/src/test/resources/org/apache/slider/core/conf/examples/app_configuration.json
index 489acda..5690225 100644
--- a/slider-core/src/test/resources/org/apache/slider/core/conf/examples/app_configuration.json
+++ b/slider-core/src/test/resources/org/apache/slider/core/conf/examples/app_configuration.json
@@ -1,6 +1,8 @@
 {
   "schema": "http://example.org/specification/v2.0.0",
-
+  "metadata": {
+    "description": "org/apache/slider/core/conf/examples/app_configuration.json"
+  },
   "global": {
 
     "zookeeper.port": "2181",

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0fce42fb/slider-core/src/test/resources/org/apache/slider/core/conf/examples/internal-resolved.json
----------------------------------------------------------------------
diff --git a/slider-core/src/test/resources/org/apache/slider/core/conf/examples/internal-resolved.json b/slider-core/src/test/resources/org/apache/slider/core/conf/examples/internal-resolved.json
index 592b4dc..da53b94 100644
--- a/slider-core/src/test/resources/org/apache/slider/core/conf/examples/internal-resolved.json
+++ b/slider-core/src/test/resources/org/apache/slider/core/conf/examples/internal-resolved.json
@@ -2,7 +2,7 @@
   "schema": "http://example.org/specification/v2.0.0",
 
   "metadata": {
-    "description": "Internal configuration DO NOT EDIT"
+    "description": "Internal resolved - org/apache/slider/core/conf/examples/internal-resolved.json"
   },
   "global": {
     "application.name": "small_cluster",

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0fce42fb/slider-core/src/test/resources/org/apache/slider/core/conf/examples/internal.json
----------------------------------------------------------------------
diff --git a/slider-core/src/test/resources/org/apache/slider/core/conf/examples/internal.json b/slider-core/src/test/resources/org/apache/slider/core/conf/examples/internal.json
index 4c782fb..b628d10 100644
--- a/slider-core/src/test/resources/org/apache/slider/core/conf/examples/internal.json
+++ b/slider-core/src/test/resources/org/apache/slider/core/conf/examples/internal.json
@@ -2,7 +2,7 @@
   "schema": "http://example.org/specification/v2.0.0",
 
   "metadata": {
-    "description": "Internal configuration DO NOT EDIT"
+    "description": "Internal unresolved - org/apache/slider/core/conf/examples/internal.json"
   },
   "global": {
     "application.name": "small_cluster",

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0fce42fb/slider-core/src/test/resources/org/apache/slider/core/conf/examples/resources.json
----------------------------------------------------------------------
diff --git a/slider-core/src/test/resources/org/apache/slider/core/conf/examples/resources.json b/slider-core/src/test/resources/org/apache/slider/core/conf/examples/resources.json
index 9d1e916..206424d 100644
--- a/slider-core/src/test/resources/org/apache/slider/core/conf/examples/resources.json
+++ b/slider-core/src/test/resources/org/apache/slider/core/conf/examples/resources.json
@@ -2,7 +2,7 @@
   "schema": "http://example.org/specification/v2.0.0",
 
   "metadata": {
-    "description": "example of a resources file"
+    "description": "example of a resources file: org/apache/slider/core/conf/examples/resources.json"
   },
   
   "global": {


[20/22] incubator-slider git commit: SLIDER-966 initial sequence and sequence on flex up holding

Posted by st...@apache.org.
SLIDER-966 initial sequence and sequence on flex up holding


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: b54eb4a399d3daf88b11c5aea96d74ee10f852e9
Parents: aa46b47
Author: Steve Loughran <st...@apache.org>
Authored: Fri Nov 6 23:15:33 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Nov 6 23:15:33 2015 +0000

----------------------------------------------------------------------
 .../slider/server/appmaster/state/AppState.java | 28 +++++++---
 .../server/appmaster/state/RoleStatus.java      | 22 ++++++--
 .../appstate/TestMockAppStateAAPlacement.groovy | 56 +++++++++++++++-----
 .../model/mock/BaseMockAppStateTest.groovy      |  4 +-
 .../appmaster/model/mock/MockYarnEngine.groovy  |  7 ++-
 5 files changed, 89 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b54eb4a3/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 1e23bef..53ab2fe 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
@@ -18,7 +18,6 @@
 
 package org.apache.slider.server.appmaster.state;
 
-import com.codahale.metrics.Counter;
 import com.codahale.metrics.Metric;
 import com.codahale.metrics.MetricRegistry;
 import com.google.common.annotations.VisibleForTesting;
@@ -1187,7 +1186,6 @@ public class AppState {
     roleHistory.onContainerReleaseSubmitted(container);
   }
 
-
   /**
    * Create a container request.
    * Update internal state, such as the role request count. 
@@ -1199,7 +1197,9 @@ public class AppState {
   private AMRMClient.ContainerRequest createContainerRequest(RoleStatus role) {
     incrementRequestCount(role);
     OutstandingRequest request = roleHistory.requestContainerForRole(role);
-    role.setOutstandingAArequest(request);
+    if (role.isAntiAffinePlacement()) {
+      role.setOutstandingAArequest(request);
+    }
     return request.getIssuedRequest();
   }
 
@@ -1772,8 +1772,10 @@ public class AppState {
       throws TriggerClusterTeardownException {
     long failures = role.getFailedRecently();
     int threshold = getFailureThresholdForRole(role);
-    log.debug("Failure count of component: {}: {}, threshold={}",
-        role.getName(), failures, threshold);
+    if (log.isDebugEnabled() && failures > 0) {
+      log.debug("Failure count of component: {}: {}, threshold={}",
+          role.getName(), failures, threshold);
+    }
 
     if (failures > threshold) {
       throw new TriggerClusterTeardownException(
@@ -1885,8 +1887,9 @@ public class AppState {
       if (isAA) {
         // build one only if there is none outstanding
         if (role.getPendingAntiAffineRequests() == 0) {
-          log.info("Starting an anti-affine request sequence");
-          role.incPendingAntiAffineRequests(delta);
+          log.info("Starting an anti-affine request sequence for {} nodes", delta);
+          // log the number outstanding
+          role.incPendingAntiAffineRequests(delta - 1);
           addContainerRequest(operations, createContainerRequest(role));
         } else {
           log.info("Adding {} more anti-affine requests", delta);
@@ -2110,6 +2113,17 @@ public class AppState {
       // add all requests to the operations list
       operations.addAll(allocation.operations);
 
+      // now for AA requests, add some more
+      if (role.isAntiAffinePlacement())  {
+        role.completeOutstandingAARequest();
+        if (role.getPendingAntiAffineRequests() > 0) {
+          // still an outstanding AA request: need to issue a new one.
+          log.info("Asking for next container for AA role {}", roleName);
+          role.decPendingAntiAffineRequests();
+          addContainerRequest(operations, createContainerRequest(role));
+        }
+      }
+
       //look for condition where we get more back than we asked
       if (allocated > desired) {
         log.info("Discarding surplus {} container {} on {}", roleName,  cid,

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b54eb4a3/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
index cba963c..1beaddc 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
@@ -28,7 +28,6 @@ import org.apache.slider.server.appmaster.management.LongGauge;
 import java.io.Serializable;
 import java.util.Comparator;
 import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
  * Models the ongoing status of all nodes in an application.
@@ -298,6 +297,10 @@ public final class RoleStatus implements Cloneable {
     this.pendingAntiAffineRequests.set(pendingAntiAffineRequests);
   }
 
+  public long decPendingAntiAffineRequests() {
+    return pendingAntiAffineRequests.decToFloor(1);
+  }
+
   public OutstandingRequest getOutstandingAArequest() {
     return outstandingAArequest;
   }
@@ -307,6 +310,15 @@ public final class RoleStatus implements Cloneable {
   }
 
   /**
+   * Complete the outstanding AA request (there's no check for one in progress, caller
+   * expected to have done that).
+   * @return the number of outstanding requests
+   */
+  public void completeOutstandingAARequest() {
+    setOutstandingAArequest(null);
+  }
+
+  /**
    * Get the number of roles we are short of.
    * nodes released are ignored.
    * @return the positive or negative number of roles to add/release.
@@ -326,11 +338,11 @@ public final class RoleStatus implements Cloneable {
   }
 
   /**
-   * Get count of actual and requested containers
-   * @return the size of the application when outstanding requests are included
+   * Get count of actual and requested containers. This includes pending ones
+   * @return the size of the application when outstanding requests are included.
    */
   public long getActualAndRequested() {
-    return actual.get() + requested.get();
+    return actual.get() + requested.get() + pendingAntiAffineRequests.get();
   }
 
   @Override
@@ -342,7 +354,7 @@ public final class RoleStatus implements Cloneable {
            ", actual=" + actual +
            ", requested=" + requested +
            ", releasing=" + releasing +
-           ", pendingAntiAffineRequestCount=" + pendingAntiAffineRequests +
+           ", pendingAntiAffineRequests=" + pendingAntiAffineRequests +
            ", failed=" + failed +
            ", failed recently=" + failedRecently.get() +
            ", node failed=" + nodeFailed.get() +

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b54eb4a3/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 42772c5..928e355 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
@@ -22,19 +22,16 @@ import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.yarn.api.records.Container
 import org.apache.hadoop.yarn.client.api.AMRMClient
-import org.apache.slider.api.ResourceKeys
 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.MockRoles
 import org.apache.slider.server.appmaster.operations.AbstractRMOperation
-import org.apache.slider.server.appmaster.operations.CancelSingleRequest
-import org.apache.slider.server.appmaster.operations.ContainerRequestOperation
 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
 
 /**
@@ -56,6 +53,8 @@ class TestMockAppStateAAPlacement extends BaseMockAppStateTest
       2,
       null)
 
+  RoleStatus aaRole
+
   @Override
   AppStateBindingInfo buildBindingInfo() {
     def bindingInfo = super.buildBindingInfo()
@@ -67,7 +66,11 @@ class TestMockAppStateAAPlacement extends BaseMockAppStateTest
     bindingInfo
   }
 
-  private static final int roleId = AAROLE.id
+  @Override
+  void setup() {
+    super.setup()
+    aaRole = lookupRole(AAROLE.name)
+  }
 
   /**
    * Get the single request of a list of operations; includes the check for the size
@@ -80,18 +83,12 @@ class TestMockAppStateAAPlacement extends BaseMockAppStateTest
   }
 
   @Test
-  public void testVerifyNodeMap() throws Throwable {
-
+  public void testAllocateAANoLabel() throws Throwable {
     def nodemap = appState.roleHistory.cloneNodemap()
     assert nodemap.size() > 0
-  }
 
-  @Test
-  public void testAllocateAANoLabel() throws Throwable {
 
-    def aaRole = lookupRole(AAROLE.name)
-
-    // want two instances, so there will be two iterations
+    // want multiple instances, so there will be iterations
     aaRole.desired = 2
 
     List<AbstractRMOperation> ops = appState.reviewRequestAndReleaseNodes()
@@ -118,6 +115,9 @@ class TestMockAppStateAAPlacement extends BaseMockAppStateTest
     // we also expect a new allocation request to have been issued
 
     def req2 = getRequest(operations, 1)
+
+    // verify the pending couner is down
+    assert 0L == aaRole.pendingAntiAffineRequests
     Container allocated2 = engine.allocateContainer(req2)
 
     // placement must be on a different host
@@ -131,7 +131,37 @@ class TestMockAppStateAAPlacement extends BaseMockAppStateTest
     assert appState.onNodeManagerContainerStarted(container.id)
     ops = appState.reviewRequestAndReleaseNodes()
     assert ops.size() == 0
+  }
+
+  @Test
+  public void testAllocateFlexUp() throws Throwable {
+    // want multiple instances, so there will be iterations
+    aaRole.desired = 2
+    List<AbstractRMOperation> ops = appState.reviewRequestAndReleaseNodes()
+    getSingleRequest(ops)
+    assert aaRole.pendingAntiAffineRequests == 1
+
+    // now trigger that flex up
+    aaRole.desired = 3
+
+    // expect: no new reqests, pending count ++
+    List<AbstractRMOperation> ops2 = appState.reviewRequestAndReleaseNodes()
+    assert ops2.empty
+    assert aaRole.pendingAntiAffineRequests == 2
+
+    // next iter
+    assert 1 == submitOperations(ops, [], ops2).size()
+    assert 2 == ops2.size()
+    assert aaRole.pendingAntiAffineRequests == 1
+
+
+    assert 0 == appState.reviewRequestAndReleaseNodes().size()
 
+    // now trigger the next execution cycle
+    List<AbstractRMOperation> ops3 = []
+    assert 1  == submitOperations(ops2, [], ops3).size()
+    assert 2 == ops3.size()
+    assert aaRole.pendingAntiAffineRequests == 0
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b54eb4a3/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
index cefba42..14e556a 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
@@ -286,9 +286,9 @@ abstract class BaseMockAppStateTest extends SliderTestBase implements MockRoles
    * @return a list of roles allocated
    */
   public List<RoleInstance> createAndSubmitNodes(
-      List<ContainerId> released) {
+      List<ContainerId> containerIds) {
     List<AbstractRMOperation> ops = appState.reviewRequestAndReleaseNodes()
-    return submitOperations(ops, released)
+    return submitOperations(ops, containerIds)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b54eb4a3/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnEngine.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnEngine.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnEngine.groovy
index 965219d..7ab97fa 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnEngine.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockYarnEngine.groovy
@@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.api.records.Container
 import org.apache.hadoop.yarn.api.records.ContainerId
 import org.apache.hadoop.yarn.client.api.AMRMClient
 import org.apache.slider.server.appmaster.operations.AbstractRMOperation
+import org.apache.slider.server.appmaster.operations.CancelSingleRequest
 import org.apache.slider.server.appmaster.operations.ContainerReleaseOperation
 import org.apache.slider.server.appmaster.operations.ContainerRequestOperation
 import org.junit.Assert
@@ -113,7 +114,9 @@ class MockYarnEngine {
         ContainerId cid = cro.containerId
         assert releaseContainer(cid);
         released.add(cid)
-      } else {
+      } else if (op instanceof CancelSingleRequest) {
+        // no-op
+      } else if (op instanceof ContainerRequestOperation) {
         ContainerRequestOperation req = (ContainerRequestOperation) op
         Container container = allocateContainer(req.request)
         if (container != null) {
@@ -123,6 +126,8 @@ class MockYarnEngine {
           log.debug("Unsatisfied allocation $req")
           pending.add(req)
         }
+      } else {
+        log.warn("Unsupported operation $op")
       }
     }
     return allocation


[08/22] incubator-slider git commit: SLIDER-963 Write mock/unit tests for AA placement

Posted by st...@apache.org.
SLIDER-963 Write mock/unit tests for AA placement


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: a3591eb16408213927b01f3f33c11bd016f13a87
Parents: 55d50fe
Author: Steve Loughran <st...@apache.org>
Authored: Thu Nov 5 18:16:43 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Nov 5 18:16:43 2015 +0000

----------------------------------------------------------------------
 .../apache/slider/common/tools/Comparators.java | 13 ++++-
 .../apache/slider/common/tools/SliderUtils.java |  1 +
 .../server/appmaster/state/NodeEntry.java       |  7 +++
 .../server/appmaster/state/NodeInstance.java    | 32 +++++------
 .../appmaster/state/OutstandingRequest.java     |  3 +-
 .../server/appmaster/state/RoleHistory.java     |  2 +-
 .../server/appmaster/state/RoleStatus.java      |  3 ++
 .../TestMockAppStateContainerFailure.groovy     |  6 +--
 .../history/TestRoleHistoryNIComparators.groovy | 45 +++++++++-------
 ...tRoleHistoryOutstandingRequestTracker.groovy |  7 +--
 .../model/history/TestRoleHistoryRW.groovy      |  4 +-
 .../history/TestRoleHistoryRWOrdering.groovy    |  8 +--
 .../TestRoleHistoryRequestTracking.groovy       | 57 +++++++++++---------
 .../model/mock/BaseMockAppStateTest.groovy      |  4 ++
 14 files changed, 112 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a3591eb1/slider-core/src/main/java/org/apache/slider/common/tools/Comparators.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/tools/Comparators.java b/slider-core/src/main/java/org/apache/slider/common/tools/Comparators.java
index 0ccca0f..6380d0c 100644
--- a/slider-core/src/main/java/org/apache/slider/common/tools/Comparators.java
+++ b/slider-core/src/main/java/org/apache/slider/common/tools/Comparators.java
@@ -21,6 +21,9 @@ package org.apache.slider.common.tools;
 import java.io.Serializable;
 import java.util.Comparator;
 
+/**
+ * Some general comparators
+ */
 public class Comparators {
 
   public static class LongComparator implements Comparator<Long>, Serializable {
@@ -30,12 +33,20 @@ public class Comparators {
       // need to comparisons with a diff greater than integer size
       if (result < 0 ) {
         return -1;
-      } else if (result >0) {
+      } else if (result > 0) {
         return 1;
       }
       return 0;
     }
   }
+public static class InvertedLongComparator implements Comparator<Long>, Serializable {
+  private static final LongComparator inner = new LongComparator();
+    @Override
+    public int compare(Long o1, Long o2) {
+      return -inner.compare(o1, o2);
+    }
+  }
+
 
   /**
    * Little template class to reverse any comparitor

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a3591eb1/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
index 1f97982..76668bf 100644
--- a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
+++ b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
@@ -2482,6 +2482,7 @@ public final class SliderUtils {
   }
 
   public static String requestToString(AMRMClient.ContainerRequest request) {
+    Preconditions.checkArgument(request != null, "Null request");
     StringBuffer buffer = new StringBuffer(request.toString());
     buffer.append("; ");
     buffer.append("relaxLocality=").append(request.getRelaxLocality()).append("; ");

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a3591eb1/slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeEntry.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeEntry.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeEntry.java
index 8ff0895..6dae3c6 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeEntry.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeEntry.java
@@ -18,6 +18,7 @@
 
 package org.apache.slider.server.appmaster.state;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.slider.api.types.NodeEntryInformation;
 
 /**
@@ -256,10 +257,16 @@ public class NodeEntry {
     return failedRecently;
   }
 
+  @VisibleForTesting
+  public synchronized void setFailedRecently(int failedRecently) {
+    this.failedRecently = failedRecently;
+  }
+
   public synchronized int getPreempted() {
     return preempted;
   }
 
+
   /**
    * Reset the failed recently count.
    */

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a3591eb1/slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java
index 2afdc42..7fc912d 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.slider.api.types.NodeEntryInformation;
 import org.apache.slider.api.types.NodeInformation;
+import org.apache.slider.common.tools.Comparators;
 import org.apache.slider.common.tools.SliderUtils;
 
 import java.io.Serializable;
@@ -69,7 +70,7 @@ public class NodeInstance {
    * @param report latest node report
    * @return true if the node state changed
    */
-  public boolean updateNode(NodeReport report) {
+  public synchronized boolean updateNode(NodeReport report) {
     nodeReport = report;
     NodeState oldState = nodeState;
     nodeState = report.getNodeState();
@@ -272,16 +273,18 @@ public class NodeInstance {
 
   /**
    * A comparator for sorting entries where the node is preferred over another.
-   * <p>
-   * The exact algorithm may change
-   * 
+   *
+   * The exact algorithm may change: current policy is "most recent first", so sorted
+   * on the lastUsed
+   *
    * the comparision is a positive int if left is preferred to right;
    * negative if right over left, 0 for equal
    */
-  public static class Preferred implements Comparator<NodeInstance>,
-                                           Serializable {
+  public static class Preferred implements Comparator<NodeInstance>, Serializable {
 
-    final int role;
+    private static final Comparators.InvertedLongComparator comparator =
+        new Comparators.InvertedLongComparator();
+    private final int role;
 
     public Preferred(int role) {
       this.role = role;
@@ -291,16 +294,9 @@ public class NodeInstance {
     public int compare(NodeInstance o1, NodeInstance o2) {
       NodeEntry left = o1.get(role);
       NodeEntry right = o2.get(role);
-      long ageL = left != null ? left.getLastUsed() : 0;
-      long ageR = right != null ? right.getLastUsed() : 0;
-      
-      if (ageL > ageR) {
-        return -1;
-      } else if (ageL < ageR) {
-        return 1;
-      }
-      // equal
-      return 0;
+      long ageL = left != null ? left.getLastUsed() : -1;
+      long ageR = right != null ? right.getLastUsed() : -1;
+      return comparator.compare(ageL, ageR);
     }
   }
 
@@ -313,7 +309,7 @@ public class NodeInstance {
   public static class MoreActiveThan implements Comparator<NodeInstance>,
                                            Serializable {
 
-    final int role;
+    private final int role;
 
     public MoreActiveThan(int role) {
       this.role = role;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a3591eb1/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java
index f5689dd..602b48e 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java
@@ -353,7 +353,8 @@ public final class OutstandingRequest {
     sb.append(", mayEscalate=").append(mayEscalate);
     sb.append(", escalated=").append(escalated);
     sb.append(", escalationTimeoutMillis=").append(escalationTimeoutMillis);
-    sb.append(", issuedRequest=").append(SliderUtils.requestToString(issuedRequest));
+    sb.append(", issuedRequest=").append(
+        issuedRequest != null ? SliderUtils.requestToString(issuedRequest) : "(null)");
     sb.append('}');
     return sb.toString();
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a3591eb1/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
index 53c2689..6880b69 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
@@ -955,7 +955,7 @@ public class RoleHistory {
    * @return a clone of the list
    */
   @VisibleForTesting
-  public List<NodeInstance> cloneAvailableList(int role) {
+  public List<NodeInstance> cloneRecentNodeList(int role) {
     return new LinkedList<>(listRecentNodesForRoleId(role));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a3591eb1/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
index 98a8311..52df406 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
@@ -61,6 +61,9 @@ public final class RoleStatus implements Cloneable {
    */
   private int pendingAntiAffineRequestCount = 0;
 
+  /** any pending AA request */
+  public OutstandingRequest outstandingAArequest = null;
+
   private String failureMessage = "";
 
   public RoleStatus(ProviderRole providerRole) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a3591eb1/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateContainerFailure.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateContainerFailure.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateContainerFailure.groovy
index 1b79115..5b24a59 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateContainerFailure.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateContainerFailure.groovy
@@ -95,7 +95,7 @@ class TestMockAppStateContainerFailure extends BaseMockAppStateTest
 
     //view the world
     appState.getRoleHistory().dump();
-    List<NodeInstance> queue = appState.roleHistory.cloneAvailableList(0)
+    List<NodeInstance> queue = appState.roleHistory.cloneRecentNodeList(0)
     assert queue.size() == 0
 
   }
@@ -123,7 +123,7 @@ class TestMockAppStateContainerFailure extends BaseMockAppStateTest
 
     //view the world
     appState.getRoleHistory().dump();
-    List<NodeInstance> queue = appState.roleHistory.cloneAvailableList(0)
+    List<NodeInstance> queue = appState.roleHistory.cloneRecentNodeList(0)
     assert queue.size() == 1
 
   }
@@ -148,7 +148,7 @@ class TestMockAppStateContainerFailure extends BaseMockAppStateTest
     
     RoleHistory history = appState.roleHistory
     history.dump();
-    List<NodeInstance> queue = history.cloneAvailableList(0)
+    List<NodeInstance> queue = history.cloneRecentNodeList(0)
     assert queue.size() == 0
 
     NodeInstance ni = history.getOrCreateNodeInstance(instance.container)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a3591eb1/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryNIComparators.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryNIComparators.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryNIComparators.groovy
index b26b2f0..ee910e4 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryNIComparators.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryNIComparators.groovy
@@ -18,6 +18,7 @@
 
 package org.apache.slider.server.appmaster.model.history
 
+import groovy.transform.CompileStatic
 import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
 import org.apache.slider.server.appmaster.model.mock.MockFactory
 import org.apache.slider.server.appmaster.state.NodeInstance
@@ -27,17 +28,19 @@ import org.junit.Test
 /**
  * Unit test to verify the comparators sort as expected
  */
+@CompileStatic
 class TestRoleHistoryNIComparators extends BaseMockAppStateTest  {
 
-  NodeInstance age1Active4 = nodeInstance(1000, 4, 0, 0)
-  NodeInstance age2Active2 = nodeInstance(1001, 2, 0, 0)
-  NodeInstance age3Active0 = nodeInstance(1002, 0, 0, 0)
-  NodeInstance age4Active1 = nodeInstance(1005, 0, 0, 0)
+  NodeInstance age1Active4 = nodeInstance(1001, 4, 0, 0)
+  NodeInstance age2Active2 = nodeInstance(1002, 2, 0, 0)
+  NodeInstance age3Active0 = nodeInstance(1003, 0, 0, 0)
+  NodeInstance age4Active1 = nodeInstance(1004, 1, 0, 0)
   NodeInstance empty = new NodeInstance("empty", MockFactory.ROLE_COUNT)
   NodeInstance age6failing = nodeInstance(1006, 0, 0, 0)
-  NodeInstance age1failing = nodeInstance(1000, 0, 0, 0)
+  NodeInstance age1failing = nodeInstance(1001, 0, 0, 0)
 
   List<NodeInstance> nodes = [age2Active2, age4Active1, age1Active4, age3Active0]
+  List<NodeInstance> allnodes = [age6failing, age2Active2, age4Active1, age1Active4, age3Active0, age1failing]
 
   @Before
   public void setup() {
@@ -51,43 +54,49 @@ class TestRoleHistoryNIComparators extends BaseMockAppStateTest  {
   }
 
   @Test
-  public void testNewerThan() throws Throwable {
-
+  public void testPreferred() throws Throwable {
     Collections.sort(nodes, new NodeInstance.Preferred(0))
-    assertListEquals(nodes,
-                     [age4Active1, age3Active0, age2Active2, age1Active4])
+    assertListEquals(nodes, [age4Active1, age3Active0, age2Active2, age1Active4])
+  }
+
+  /**
+   * The preferred sort still includes failures; up to next phase in process
+   * to handle that
+   * @throws Throwable
+   */
+  @Test
+  public void testPreferredWithFailures() throws Throwable {
+    Collections.sort(allnodes, new NodeInstance.Preferred(0))
+    assert allnodes[0] == age6failing
+    assert allnodes[1] == age4Active1
   }
 
   @Test
-  public void testFailureCountFirst() throws Throwable {
+  public void testPreferredComparatorDowngradesFailures() throws Throwable {
     def preferred = new NodeInstance.Preferred(0)
     assert preferred.compare(age6failing, age1failing) == -1
     assert preferred.compare(age1failing, age6failing) == 1
   }
-  
+
   @Test
   public void testNewerThanNoRole() throws Throwable {
-
     nodes << empty
     Collections.sort(nodes, new NodeInstance.Preferred(0))
-    assertListEquals(nodes,
-                     [age4Active1, age3Active0, age2Active2, age1Active4, empty])
+    assertListEquals(nodes, [age4Active1, age3Active0, age2Active2, age1Active4, empty])
   }
 
   @Test
   public void testMoreActiveThan() throws Throwable {
 
     Collections.sort(nodes, new NodeInstance.MoreActiveThan(0))
-    assertListEquals(nodes,
-                     [age1Active4, age2Active2, age4Active1, age3Active0],)
+    assertListEquals(nodes, [age1Active4, age2Active2, age4Active1, age3Active0])
   }
 
   @Test
   public void testMoreActiveThanEmpty() throws Throwable {
     nodes << empty
     Collections.sort(nodes, new NodeInstance.MoreActiveThan(0))
-    assertListEquals(nodes,
-                     [age1Active4, age2Active2, age4Active1, age3Active0, empty])
+    assertListEquals(nodes, [age1Active4, age2Active2, age4Active1, age3Active0, empty])
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a3591eb1/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
index 653af84..33cacd7 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
@@ -35,17 +35,12 @@ import org.apache.slider.server.appmaster.state.OutstandingRequestTracker
 import org.apache.slider.server.appmaster.state.RoleStatus
 import org.junit.Test
 
-class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest {
+class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
 
   NodeInstance host1 = new NodeInstance("host1", 3)
   NodeInstance host2 = new NodeInstance("host2", 3)
 
   OutstandingRequestTracker tracker = new OutstandingRequestTracker()
-  
-  @Override
-  String getTestName() {
-    return "TestOutstandingRequestTracker"
-  }
 
   @Test
   public void testAddRetrieveEntry() throws Throwable {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a3591eb1/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
index 7afcfc1..a1e424f 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
@@ -137,13 +137,13 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
     rh2.buildRecentNodeLists();
     describe("starting")
     rh2.dump();
-    List<NodeInstance> available0 = rh2.cloneAvailableList(0)
+    List<NodeInstance> available0 = rh2.cloneRecentNodeList(0)
     assert available0.size() == 1
 
     NodeInstance entry = available0.get(0)
     assert entry.hostname == "localhost"
     assert entry == localhost
-    List<NodeInstance> available1 = rh2.cloneAvailableList(1)
+    List<NodeInstance> available1 = rh2.cloneRecentNodeList(1)
     assert available1.size() == 2
     //and verify that even if last used was set, the save time is picked up
     assert entry.get(1).lastUsed == roleHistory.saveTime

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a3591eb1/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRWOrdering.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRWOrdering.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRWOrdering.groovy
index aef22fb..0655531 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRWOrdering.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRWOrdering.groovy
@@ -18,6 +18,7 @@
 
 package org.apache.slider.server.appmaster.model.history
 
+import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.fs.Path
 import org.apache.slider.common.SliderKeys
@@ -34,9 +35,10 @@ import java.util.regex.Matcher
 import java.util.regex.Pattern
 
 @Slf4j
+@CompileStatic
 class TestRoleHistoryRWOrdering extends BaseMockAppStateTest {
 
-  def paths = pathlist(
+  List<Path> paths = pathlist(
       [
           "hdfs://localhost/history-0406c.json",
           "hdfs://localhost/history-5fffa.json",
@@ -60,7 +62,6 @@ class TestRoleHistoryRWOrdering extends BaseMockAppStateTest {
     return "TestHistoryRWOrdering"
   }
 
-    
   /**
    * This tests regexp pattern matching. It uses the current time so isn't
    * repeatable -but it does test a wider range of values in the process
@@ -79,7 +80,6 @@ class TestRoleHistoryRWOrdering extends BaseMockAppStateTest {
     }
   }
 
-
   @Test
   public void testWriteSequenceReadData() throws Throwable {
     describe "test that if multiple entries are written, the newest is picked up"
@@ -116,7 +116,7 @@ class TestRoleHistoryRWOrdering extends BaseMockAppStateTest {
   public void testPathStructure() throws Throwable {
     assert h_5fffa.getName() == "history-5fffa.json"
   }
-  
+
   @Test
   public void testPathnameComparator() throws Throwable {
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a3591eb1/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
index db795d0..42d0c50 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
@@ -18,6 +18,7 @@
 
 package org.apache.slider.server.appmaster.model.history
 
+import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.yarn.api.records.Container
 import org.apache.hadoop.yarn.api.records.Resource
@@ -41,6 +42,7 @@ import org.junit.Test
  * get removed and added 
  */
 @Slf4j
+@CompileStatic
 class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
 
   String roleName = "test"
@@ -54,6 +56,7 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
 
   List<NodeInstance> nodes = [age2Active2, age2Active0, age4Active1, age1Active4, age3Active0]
   RoleHistory roleHistory = new RoleHistory(MockFactory.ROLES)
+  /** 1MB, 1 vcore*/
   Resource resource = Resource.newInstance(1, 1)
 
   ProviderRole provRole = new ProviderRole(roleName, 0)
@@ -72,27 +75,25 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
 
   @Test
   public void testAvailableListBuiltForRoles() throws Throwable {
-    List<NodeInstance> available0 = roleHistory.cloneAvailableList(0)
+    List<NodeInstance> available0 = roleHistory.cloneRecentNodeList(0)
     assertListEquals([age3Active0, age2Active0], available0)
   }
 
   @Test
   public void testRequestedNodeOffList() throws Throwable {
-    List<NodeInstance> available0 = roleHistory.cloneAvailableList(0)
     NodeInstance ni = roleHistory.findNodeForNewInstance(roleStatus)
     assert age3Active0 == ni
-    AMRMClient.ContainerRequest req = roleHistory.requestInstanceOnNode(ni,
+    assertListEquals([age2Active0], roleHistory.cloneRecentNodeList(0))
+    roleHistory.requestInstanceOnNode(ni,
         roleStatus,
         resource,
         "")
-    List<NodeInstance> a2 = roleHistory.cloneAvailableList(0)
-    assertListEquals([age2Active0], a2)
   }
 
   @Test
   public void testRequestedNodeOffListWithFailures() throws Throwable {
     assert 0 == roleStatus.key
-    assert !roleHistory.cloneAvailableList(0).isEmpty()
+    assert !roleHistory.cloneRecentNodeList(0).empty
 
     NodeEntry age3role0 = recordAsFailed(age3Active0, 0, 4)
     assert age3Active0.isConsideredUnreliable(0, roleStatus.nodeFailureThreshold)
@@ -115,51 +116,55 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
     roleHistory.dump()
     assert 0 == age3role0.failedRecently
     assert !age3Active0.isConsideredUnreliable(0, roleStatus.nodeFailureThreshold)
-    assert !roleHistory.cloneAvailableList(0).isEmpty()
+    assert !roleHistory.cloneRecentNodeList(0).empty
     // looking for a node should now find one
     ni = roleHistory.findNodeForNewInstance(roleStatus)
     assert ni == age3Active0
-    req = roleHistory.requestInstanceOnNode(ni,
-        roleStatus,
-        resource,
-        "")
+    req = roleHistory.requestInstanceOnNode(ni, roleStatus, resource, "")
     assert 1 == req.nodes.size()
   }
 
+  /**
+   * verify that strict placement policies generate requests for nodes irrespective
+   * of their failed status
+   * @throws Throwable
+   */
   @Test
   public void testStrictPlacementIgnoresFailures() throws Throwable {
 
     def targetRole = role1Status
     final ProviderRole providerRole1 = targetRole.providerRole
     assert providerRole1.placementPolicy == PlacementPolicy.STRICT
-    int key = targetRole.key
+    int key1 = targetRole.key
+    def key0 = role0Status.key
 
-    recordAsFailed(age1Active4, key, 4)
-    recordAsFailed(age2Active0, key, 4)
-    recordAsFailed(age2Active2, key, 4)
-    recordAsFailed(age3Active0, key, 4)
-    recordAsFailed(age4Active1, key, 4)
+    def nodes = [age1Active4, age2Active0, age2Active2, age3Active0, age4Active1]
+    recordAllFailed(key0, 4, nodes)
+    recordAllFailed(key1, 4, nodes)
 
     // trigger a list rebuild
     roleHistory.buildRecentNodeLists();
+    def recentRole0 = roleHistory.cloneRecentNodeList(key0)
+    assert recentRole0.indexOf(age3Active0) < recentRole0.indexOf(age2Active0)
 
-    assert !roleHistory.cloneAvailableList(key).isEmpty()
+    // the non-strict role has no suitable nodes
+    assert null == roleHistory.findNodeForNewInstance(role0Status)
 
 
-    NodeInstance ni = roleHistory.findNodeForNewInstance(targetRole)
-    assert ni == age4Active1!= null
-    // next lookup returns next node
-    ni = roleHistory.findNodeForNewInstance(roleStatus)
-    assert ni == age3Active0
-  }
+    def ni = roleHistory.findNodeForNewInstance(targetRole)
+    assert ni
 
+    def ni2 = roleHistory.findNodeForNewInstance(targetRole)
+    assert ni2
+    assert ni != ni2
+  }
 
   @Test
   public void testFindAndRequestNode() throws Throwable {
     AMRMClient.ContainerRequest req = roleHistory.requestNode(roleStatus, resource)
 
     assert age3Active0.hostname == req.nodes[0]
-    List<NodeInstance> a2 = roleHistory.cloneAvailableList(0)
+    List<NodeInstance> a2 = roleHistory.cloneRecentNodeList(0)
     assertListEquals([age2Active0], a2)
   }
 
@@ -239,7 +244,7 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
     roleHistory.listOpenRequests().empty
 
     // and the remainder goes onto the available list
-    List<NodeInstance> a2 = roleHistory.cloneAvailableList(0)
+    List<NodeInstance> a2 = roleHistory.cloneRecentNodeList(0)
     assertListEquals([age2Active0], a2)
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a3591eb1/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
index 33ea0a0..babce9b 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
@@ -356,4 +356,8 @@ abstract class BaseMockAppStateTest extends SliderTestBase implements MockRoles
     }
     entry
   }
+
+  def recordAllFailed(int id, int count, List<NodeInstance> nodes) {
+    nodes.each { NodeInstance node -> recordAsFailed(node, id, count)}
+  }
 }


[11/22] incubator-slider git commit: SLIDER-963 Write mock/unit tests for AA placement

Posted by st...@apache.org.
SLIDER-963 Write mock/unit tests for AA placement


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: 7f964166fcb8bb09d98f8c0cfce4839481ec6317
Parents: ee8a9be
Author: Steve Loughran <st...@apache.org>
Authored: Thu Nov 5 18:58:40 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Nov 5 18:58:40 2015 +0000

----------------------------------------------------------------------
 ...tRoleHistoryOutstandingRequestTracker.groovy | 36 ++++++++------------
 .../appmaster/model/mock/MockFactory.groovy     |  8 ++---
 .../appmaster/model/mock/MockResource.groovy    |  2 +-
 3 files changed, 19 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/7f964166/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
index 56b2c31..8399d53 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.groovy
@@ -39,6 +39,7 @@ class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
 
   NodeInstance host1 = new NodeInstance("host1", 3)
   NodeInstance host2 = new NodeInstance("host2", 3)
+  def resource = factory.newResource(48, 1)
 
   OutstandingRequestTracker tracker = new OutstandingRequestTracker()
 
@@ -52,9 +53,11 @@ class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
 
   @Test
   public void testAddCompleteEntry() throws Throwable {
-    tracker.newRequest(host1, 0)
-    tracker.newRequest(host2, 0)
-    tracker.newRequest(host1, 1)
+    def req1 = tracker.newRequest(host1, 0)
+    req1.buildContainerRequest(resource, role0Status, 0, "")
+
+    tracker.newRequest(host2, 0).buildContainerRequest(resource, role0Status, 0, "")
+    tracker.newRequest(host1, 1).buildContainerRequest(resource, role0Status, 0, "")
 
     def allocation = tracker.onContainerAllocated(1, "host1", null)
     assert allocation.outcome == ContainerAllocationOutcome.Placed
@@ -79,14 +82,10 @@ class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
   @Test
   public void testRemoveOpenRequestUnissued() throws Throwable {
     def req1 = tracker.newRequest(null, 0)
+    req1.buildContainerRequest(resource, role0Status, 0, "")
     assert tracker.listOpenRequests().size() == 1
-    def c1 = factory.newContainer()
-    c1.setPriority(new MockPriority(0))
-
-    def resource = factory.newResource()
-    resource.virtualCores=1
-    resource.memory = 48;
-    c1.setResource(resource)
+    def c1 = factory.newContainer(null, new MockPriority(0))
+    c1.resource = resource
 
     def allocation = tracker.onContainerAllocated(0, "host1", c1)
     ContainerAllocationOutcome outcome = allocation.outcome
@@ -98,26 +97,19 @@ class TestRoleHistoryOutstandingRequestTracker extends BaseMockAppStateTest  {
   @Test
   public void testIssuedOpenRequest() throws Throwable {
     def req1 = tracker.newRequest(null, 0)
-    def resource = factory.newResource()
-    resource.virtualCores = 1
-    resource.memory = 48;
-    def yarnRequest = req1.buildContainerRequest(resource, role0Status, 0, "")
+    req1.buildContainerRequest(resource, role0Status, 0, "")
     assert tracker.listOpenRequests().size() == 1
-    def c1 = factory.newContainer()
-
-    def nodeId = factory.newNodeId()
-    c1.nodeId = nodeId
-    nodeId.host ="hostname-1"
 
     def pri = ContainerPriority.buildPriority(0, false)
     assert pri > 0
-    c1.setPriority(new MockPriority(pri))
+    def nodeId = factory.newNodeId("hostname-1")
+    def c1 = factory.newContainer(nodeId, new MockPriority(pri))
 
-    c1.setResource(resource)
+    c1.resource = resource
 
     def issued = req1.issuedRequest
     assert issued.capability == resource
-    assert issued.priority.priority == c1.getPriority().getPriority()
+    assert issued.priority.priority == c1.priority.priority
     assert req1.resourceRequirementsMatch(resource)
 
     def allocation = tracker.onContainerAllocated(0, nodeId.host, c1)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/7f964166/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 25fdd8b..f7fd641 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
@@ -107,8 +107,8 @@ class MockFactory implements MockRoles {
     return id;
   }
 
-  MockNodeId newNodeId() {
-    new MockNodeId()
+  MockNodeId newNodeId(String host = null) {
+    new MockNodeId(host: host)
   }
 
   MockContainer newContainer(ContainerId cid) {
@@ -202,8 +202,8 @@ class MockFactory implements MockRoles {
     ]
   }
 
-  MockResource newResource() {
-    return new MockResource()
+  MockResource newResource(int memory = 0, int vcores = 0) {
+    return new MockResource(memory, vcores)
   }
 
   MockContainerStatus newContainerStatus() {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/7f964166/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockResource.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockResource.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockResource.groovy
index f4c54f3..91ab43b 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockResource.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockResource.groovy
@@ -24,7 +24,7 @@ class MockResource extends Resource {
   int memory
   int virtualCores
 
-  MockResource(int memory=0, int vcores=0) {
+  MockResource(int memory = 0, int vcores = 0) {
     this.memory = memory
     this.virtualCores = vcores
   }


[17/22] incubator-slider git commit: SLIDER-965 RoleHistory to (carefully) share RoleStatus instances with AppState

Posted by st...@apache.org.
SLIDER-965 RoleHistory to (carefully) share RoleStatus instances with AppState


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: 5b7f6dde5cde1856c6a870db024020619b7b0d94
Parents: 8f2786c
Author: Steve Loughran <st...@apache.org>
Authored: Fri Nov 6 17:57:59 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Nov 6 17:57:59 2015 +0000

----------------------------------------------------------------------
 .../slider/core/conf/ConfTreeOperations.java    |   9 ++
 .../server/appmaster/management/LongGauge.java  |  64 +++-----
 .../slider/server/appmaster/state/AppState.java |  66 ++++-----
 .../appmaster/state/ContainerAllocation.java    |  50 -------
 .../state/ContainerAllocationResults.java       |  50 +++++++
 .../state/ContainerReleaseSelector.java         |   5 +-
 .../MostRecentContainerReleaseSelector.java     |   3 +-
 .../state/OutstandingRequestTracker.java        |   4 +-
 .../server/appmaster/state/RoleHistory.java     |  76 +++++-----
 .../server/appmaster/state/RoleStatus.java      | 147 ++++++++++---------
 .../appmaster/state/SimpleReleaseSelector.java  |   3 +-
 .../appmaster/web/view/ContainerStatsBlock.java |   3 +-
 .../appstate/TestMockAppStateAAPlacement.groovy |  42 +-----
 .../TestMockAppStateDynamicHistory.groovy       |  16 +-
 .../TestMockAppStateDynamicRoles.groovy         |  39 ++---
 .../TestRoleHistoryContainerEvents.groovy       |   3 +-
 ...stRoleHistoryFindNodesForNewInstances.groovy |   3 +-
 .../history/TestRoleHistoryNIComparators.groovy |  12 +-
 .../model/history/TestRoleHistoryRW.groovy      |  29 ++--
 .../history/TestRoleHistoryRWOrdering.groovy    |  25 ++--
 .../TestRoleHistoryRequestTracking.groovy       |   3 +-
 .../model/mock/BaseMockAppStateTest.groovy      |  93 ++++++++----
 .../appmaster/model/mock/MockRoleHistory.groovy |  12 +-
 23 files changed, 356 insertions(+), 401 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java b/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
index bc116e7..58896ee 100644
--- a/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
+++ b/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
@@ -432,5 +432,14 @@ public class ConfTreeOperations {
   public void setComponentOpt(String role, String option, int val) {
     setComponentOpt(role, option, Integer.toString(val));
   }
+  /**
+   * Set a long role option, creating the role if necessary
+   * @param role role name
+   * @param option option name
+   * @param val long value
+   */
+  public void setComponentOpt(String role, String option, long val) {
+    setComponentOpt(role, option, Long.toString(val));
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/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 08f61ec..72a8805 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
@@ -24,69 +24,43 @@ import com.codahale.metrics.Metric;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
- * This is a long which acts as a gauge
+ * This is a {@link AtomicLong} which acts as a metrics gauge: its state can be exposed as
+ * a management value.
+ *
  */
-public class LongGauge implements Metric, Gauge<Long> {
-
-  private final AtomicLong value;
+public class LongGauge extends AtomicLong implements Metric, Gauge<Long> {
 
   /**
    * Instantiate
    * @param val current value
    */
   public LongGauge(long val) {
-    this.value = new AtomicLong(val);
-  }
-
-  public LongGauge() {
-    this(0);
+    super(val);
   }
 
   /**
-   * Set to a new value.
-   * @param val value
+   * Instantiate with value 0
    */
-  public synchronized void set(long val) {
-    value.set(val);
-  }
-
-  public void inc() {
-    inc(1);
-  }
-
-  public void dec() {
-    dec(1);
-  }
-
-  public synchronized void inc(int delta) {
-    set(value.get() + delta);
-  }
-
-  public synchronized void dec(int delta) {
-    set(value.get() - delta);
+  public LongGauge() {
+    this(0);
   }
 
-  public long get() {
-    return value.get();
-  }
 
   @Override
   public Long getValue() {
     return get();
   }
 
-  @Override
-  public String toString() {
-   return value.toString();
-  }
-
-  @Override
-  public int hashCode() {
-    return value.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    return value.equals(obj);
+  /**
+   * Decrement to the floor of 0.
+   * There's checks to stop more than one thread being in this method at the time, but
+   * that doesn't stop other operations on the value
+   * @param delta delta
+   * @return the current value
+   */
+  public synchronized long decToFloor(long delta) {
+    long newval = Math.max(0L, get() - delta);
+    set(newval);
+    return get();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/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 29d5cde..c46177a 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
@@ -554,7 +554,7 @@ public class AppState {
 
 
     // set up the role history
-    roleHistory = new RoleHistory(roleList);
+    roleHistory = new RoleHistory(roleStatusMap.values());
     roleHistory.register(metricsAndMonitoring);
     roleHistory.onStart(binding.fs, binding.historyPath);
     // trigger first node update
@@ -665,12 +665,9 @@ public class AppState {
 
 
     //snapshot all three sectons
-    resourcesSnapshot =
-      ConfTreeOperations.fromInstance(instanceDefinition.getResources());
-    appConfSnapshot =
-      ConfTreeOperations.fromInstance(instanceDefinition.getAppConf());
-    internalsSnapshot =
-      ConfTreeOperations.fromInstance(instanceDefinition.getInternal());
+    resourcesSnapshot = ConfTreeOperations.fromInstance(instanceDefinition.getResources());
+    appConfSnapshot = ConfTreeOperations.fromInstance(instanceDefinition.getAppConf());
+    internalsSnapshot = ConfTreeOperations.fromInstance(instanceDefinition.getInternal());
     //build a new aggregate from the snapshots
     instanceDefinitionSnapshot = new AggregateConf(resourcesSnapshot.confTree,
                                                    appConfSnapshot.confTree,
@@ -681,7 +678,7 @@ public class AppState {
       ClusterDescriptionOperations.buildFromInstanceDefinition(
           instanceDefinition);
 
-//     Add the -site configuration properties
+    // Add the -site configuration properties
     for (Map.Entry<String, String> prop : clientProperties.entrySet()) {
       clusterStatusTemplate.clientProperties.put(prop.getKey(), prop.getValue());
     }
@@ -694,6 +691,7 @@ public class AppState {
    * @return a list of any dynamically added provider roles
    * (purely for testing purposes)
    */
+  @VisibleForTesting
   public synchronized List<ProviderRole> updateResourceDefinitions(ConfTree resources)
       throws BadConfigException, IOException {
     log.debug("Updating resources to {}", resources);
@@ -733,10 +731,8 @@ public class AppState {
         // skip inflexible roles, e.g AM itself
         continue;
       }
-      int currentDesired = roleStatus.getDesired();
+      long currentDesired = roleStatus.getDesired();
       String role = roleStatus.getName();
-      MapOperations comp =
-          resources.getComponent(role);
       int desiredInstanceCount = getDesiredInstanceCount(resources, role);
       if (desiredInstanceCount == 0) {
         log.info("Role {} has 0 instances specified", role);
@@ -756,12 +752,11 @@ public class AppState {
         // this is a new value
         log.info("Adding new role {}", name);
         MapOperations component = resources.getComponent(name);
-        ProviderRole dynamicRole = createDynamicProviderRole(name,
-            component);
+        ProviderRole dynamicRole = createDynamicProviderRole(name, component);
         RoleStatus roleStatus = buildRole(dynamicRole);
         roleStatus.setDesired(getDesiredInstanceCount(resources, name));
         log.info("New role {}", roleStatus);
-        roleHistory.addNewProviderRole(dynamicRole);
+        roleHistory.addNewRole(roleStatus);
         newRoles.add(dynamicRole);
       }
     }
@@ -842,8 +837,7 @@ public class AppState {
     putOwnedContainer(containerId, am);
 
     // patch up the role status
-    RoleStatus roleStatus = roleStatusMap.get(
-        (SliderKeys.ROLE_AM_PRIORITY_INDEX));
+    RoleStatus roleStatus = roleStatusMap.get(SliderKeys.ROLE_AM_PRIORITY_INDEX);
     roleStatus.setDesired(1);
     roleStatus.incActual();
     roleStatus.incStarted();
@@ -905,7 +899,7 @@ public class AppState {
    */
   public List<RoleStatus> cloneRoleStatusList() {
     Collection<RoleStatus> statuses = roleStatusMap.values();
-    List<RoleStatus> statusList = new ArrayList<RoleStatus>(statuses.size());
+    List<RoleStatus> statusList = new ArrayList<>(statuses.size());
     try {
       for (RoleStatus status : statuses) {
         statusList.add((RoleStatus)(status.clone()));
@@ -1481,9 +1475,9 @@ public class AppState {
       log.info("Container was queued for release : {}", containerId);
       Container container = containersBeingReleased.remove(containerId);
       RoleStatus roleStatus = lookupRoleStatus(container);
-      int releasing = roleStatus.decReleasing();
-      int actual = roleStatus.decActual();
-      int completedCount = roleStatus.incCompleted();
+      long releasing = roleStatus.decReleasing();
+      long actual = roleStatus.decActual();
+      long completedCount = roleStatus.incCompleted();
       log.info("decrementing role count for role {} to {}; releasing={}, completed={}",
           roleStatus.getName(),
           actual,
@@ -1620,7 +1614,7 @@ public class AppState {
    */
   public synchronized float getApplicationProgressPercentage() {
     float percentage;
-    int desired = 0;
+    long desired = 0;
     float actual = 0;
     for (RoleStatus role : getRoleStatusMap().values()) {
       desired += role.getDesired();
@@ -1866,8 +1860,8 @@ public class AppState {
   private List<AbstractRMOperation> reviewOneRole(RoleStatus role)
       throws SliderInternalStateException, TriggerClusterTeardownException {
     List<AbstractRMOperation> operations = new ArrayList<>();
-    int delta;
-    int expected;
+    long delta;
+    long expected;
     String name = role.getName();
     synchronized (role) {
       delta = role.getDelta();
@@ -1909,13 +1903,13 @@ public class AppState {
       // reduce the number expected (i.e. subtract the delta)
 
       // then pick some containers to kill
-      int excess = -delta;
+      long excess = -delta;
 
       // how many requests are outstanding
-      int outstandingRequests = role.getRequested();
+      long outstandingRequests = role.getRequested();
       if (outstandingRequests > 0) {
         // outstanding requests.
-        int toCancel = Math.min(outstandingRequests, excess);
+        int toCancel = (int)Math.min(outstandingRequests, excess);
 
         // Delegate to Role History
 
@@ -1972,13 +1966,12 @@ public class AppState {
         // ask the release selector to sort the targets
         containersToRelease =  containerReleaseSelector.sortCandidates(
             roleId,
-            containersToRelease,
-            excess);
+            containersToRelease);
 
         //crop to the excess
 
         List<RoleInstance> finalCandidates = (excess < numberAvailableForRelease) 
-            ? containersToRelease.subList(0, excess)
+            ? containersToRelease.subList(0, (int)excess)
             : containersToRelease;
 
 
@@ -2085,9 +2078,8 @@ public class AppState {
     List<Container> ordered = roleHistory.prepareAllocationList(allocatedContainers);
     log.debug("onContainersAllocated(): Total containers allocated = {}", ordered.size());
     for (Container container : ordered) {
-      String containerHostInfo = container.getNodeId().getHost()
-                                 + ":" +
-                                 container.getNodeId().getPort();
+      final NodeId nodeId = container.getNodeId();
+      String containerHostInfo = nodeId.getHost() + ":" + nodeId.getPort();
       //get the role
       final ContainerId cid = container.getId();
       final RoleStatus role = lookupRoleStatus(container);
@@ -2097,11 +2089,11 @@ public class AppState {
 
       //inc allocated count -this may need to be dropped in a moment,
       // but us needed to update the logic below
-      final int allocated = role.incActual();
-      final int desired = role.getDesired();
+      final long allocated = role.incActual();
+      final long desired = role.getDesired();
 
       final String roleName = role.getName();
-      final ContainerAllocation allocation =
+      final ContainerAllocationResults allocation =
           roleHistory.onContainerAllocated(container, desired, allocated);
       final ContainerAllocationOutcome outcome = allocation.outcome;
 
@@ -2128,8 +2120,8 @@ public class AppState {
                  " on {}:{},",
                  roleName,
                  cid,
-                 container.getNodeId().getHost(),
-                 container.getNodeId().getPort()
+                 nodeId.getHost(),
+                 nodeId.getPort()
                 );
 
         assignments.add(new ContainerAssignment(container, role, outcome));

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocation.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocation.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocation.java
deleted file mode 100644
index 6bfe8ab..0000000
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocation.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.server.appmaster.state;
-
-import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * This is just a tuple of the outcome of a container allocation
- */
-public class ContainerAllocation {
-
-  /**
-   * What was the outcome of this allocation: placed, escalated, ...
-   */
-  public ContainerAllocationOutcome outcome;
-
-  /**
-   * The outstanding request which originated this.
-   * This will be null if the outcome is {@link ContainerAllocationOutcome#Unallocated}
-   * as it wasn't expected.
-   */
-  public OutstandingRequest origin;
-
-  /**
-   * A possibly empty list of requests to add to the follow-up actions
-   */
-  public List<AbstractRMOperation> operations = new ArrayList<>(0);
-
-  public ContainerAllocation() {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationResults.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationResults.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationResults.java
new file mode 100644
index 0000000..e80639e
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationResults.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This is just a tuple of the outcome of a container allocation
+ */
+public class ContainerAllocationResults {
+
+  /**
+   * What was the outcome of this allocation: placed, escalated, ...
+   */
+  public ContainerAllocationOutcome outcome;
+
+  /**
+   * The outstanding request which originated this.
+   * This will be null if the outcome is {@link ContainerAllocationOutcome#Unallocated}
+   * as it wasn't expected.
+   */
+  public OutstandingRequest origin;
+
+  /**
+   * A possibly empty list of requests to add to the follow-up actions
+   */
+  public List<AbstractRMOperation> operations = new ArrayList<>(0);
+
+  public ContainerAllocationResults() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerReleaseSelector.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerReleaseSelector.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerReleaseSelector.java
index 0cbc134..fafbada 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerReleaseSelector.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerReleaseSelector.java
@@ -30,9 +30,8 @@ public interface ContainerReleaseSelector {
    * Given a list of candidate containers, return a sorted version of the priority
    * in which they should be released. 
    * @param candidates candidate list ... everything considered suitable
-   * @return
+   * @return the list of candidates
    */
   List<RoleInstance> sortCandidates(int roleId,
-      List<RoleInstance> candidates,
-      int minimumToSelect);
+      List<RoleInstance> candidates);
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/main/java/org/apache/slider/server/appmaster/state/MostRecentContainerReleaseSelector.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/MostRecentContainerReleaseSelector.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/MostRecentContainerReleaseSelector.java
index 9d936a1..38c5b8e 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/MostRecentContainerReleaseSelector.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/MostRecentContainerReleaseSelector.java
@@ -32,8 +32,7 @@ public class MostRecentContainerReleaseSelector implements ContainerReleaseSelec
 
   @Override
   public List<RoleInstance> sortCandidates(int roleId,
-      List<RoleInstance> candidates,
-      int minimumToSelect) {
+      List<RoleInstance> candidates) {
     Collections.sort(candidates, new newerThan());
     return candidates;
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java
index bf34d43..a791826 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java
@@ -127,13 +127,13 @@ public class OutstandingRequestTracker {
    * @param hostname hostname
    * @return the allocation outcome
    */
-  public synchronized ContainerAllocation onContainerAllocated(int role,
+  public synchronized ContainerAllocationResults onContainerAllocated(int role,
       String hostname,
       Container container) {
     final String containerDetails = SliderUtils.containerToString(container);
     log.debug("Processing allocation for role {}  on {}", role,
         containerDetails);
-    ContainerAllocation allocation = new ContainerAllocation();
+    ContainerAllocationResults allocation = new ContainerAllocationResults();
     ContainerAllocationOutcome outcome;
     OutstandingRequest request = placedRequests.remove(new OutstandingRequest(role, hostname));
     if (request != null) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
index 34340a2..c93c7f5 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
@@ -71,6 +71,8 @@ public class RoleHistory {
   protected static final Logger log =
     LoggerFactory.getLogger(RoleHistory.class);
   private final List<ProviderRole> providerRoles;
+  /** the roles in here are shared with App State */
+  private final Map<Integer, RoleStatus> roleStatusMap = new HashMap<>();
   private long startTime;
 
   /** Time when saved */
@@ -110,10 +112,17 @@ public class RoleHistory {
    */
   private Set<String> failedNodes = new HashSet<>();
 
-
-  public RoleHistory(List<ProviderRole> providerRoles) throws BadConfigException {
-    this.providerRoles = providerRoles;
-    roleSize = providerRoles.size();
+  /**
+   * Instantiate
+   * @param roles initial role list
+   * @throws BadConfigException
+   */
+  public RoleHistory(Collection<RoleStatus> roles) throws BadConfigException {
+    roleSize = roles.size();
+    providerRoles = new ArrayList<>(roleSize);
+    for (RoleStatus role : roles) {
+      addNewRole(role);
+    }
     reset();
   }
 
@@ -126,13 +135,7 @@ public class RoleHistory {
     nodemap = new NodeMap(roleSize);
     failedNodes = new HashSet<>();
     resetAvailableNodeLists();
-
     outstandingRequests = new OutstandingRequestTracker();
-
-    Map<Integer, RoleStatus> roleStats = new HashMap<>();
-    for (ProviderRole providerRole : providerRoles) {
-      checkProviderRole(roleStats, providerRole);
-    }
   }
 
   /**
@@ -148,45 +151,33 @@ public class RoleHistory {
   }
 
   /**
-   * safety check: make sure the provider role is unique amongst
+   * safety check: make sure the role is unique amongst
    * the role stats...which is extended with the new role
-   * @param roleStats role stats
-   * @param providerRole role
+   * @param roleStatus role
    * @throws ArrayIndexOutOfBoundsException
    * @throws BadConfigException
    */
-  protected void checkProviderRole(Map<Integer, RoleStatus> roleStats,
-      ProviderRole providerRole)
-    throws BadConfigException {
-    int index = providerRole.id;
+  protected void putRole(RoleStatus roleStatus) throws BadConfigException {
+    int index = roleStatus.getKey();
     if (index < 0) {
-      throw new BadConfigException("Provider " + providerRole + " id is out of range");
+      throw new BadConfigException("Provider " + roleStatus + " id is out of range");
     }
-    if (roleStats.get(index) != null) {
+    if (roleStatusMap.get(index) != null) {
       throw new BadConfigException(
-        providerRole.toString() + " id duplicates that of " +
-        roleStats.get(index));
+        roleStatus.toString() + " id duplicates that of " +
+            roleStatusMap.get(index));
     }
-    roleStats.put(index, new RoleStatus(providerRole));
+    roleStatusMap.put(index, roleStatus);
   }
 
   /**
-   * Add a new provider role to the map
-   * @param providerRole new provider role
+   * Add a new role
+   * @param roleStatus new role
    */
-  public void addNewProviderRole(ProviderRole providerRole)
-    throws BadConfigException {
-    log.debug("Validating/adding new provider role to role history: {} ",
-        providerRole);
-    Map<Integer, RoleStatus> roleStats = new HashMap<>();
-
-    for (ProviderRole role : providerRoles) {
-      roleStats.put(role.id, new RoleStatus(role));
-    }
-
-    checkProviderRole(roleStats, providerRole);
-    log.debug("Check successful; adding role");
-    this.providerRoles.add(providerRole);
+  public void addNewRole(RoleStatus roleStatus) throws BadConfigException {
+    log.debug("Validating/adding new role to role history: {} ", roleStatus);
+    putRole(roleStatus);
+    this.providerRoles.add(roleStatus.getProviderRole());
   }
 
   /**
@@ -713,13 +704,14 @@ public class RoleHistory {
    * @param actualCount current count of instances
    * @return The allocation outcome
    */
-  public synchronized ContainerAllocation onContainerAllocated(Container container,
-      int desiredCount,
-      int actualCount) {
+  public synchronized ContainerAllocationResults onContainerAllocated(Container container,
+      long desiredCount,
+      long actualCount) {
     int role = ContainerPriority.extractRole(container);
+
     String hostname = RoleHistoryUtils.hostnameOf(container);
     List<NodeInstance> nodeInstances = listRecentNodesForRoleId(role);
-    ContainerAllocation outcome =
+    ContainerAllocationResults outcome =
         outstandingRequests.onContainerAllocated(role, hostname, container);
     if (desiredCount <= actualCount) {
       // all outstanding requests have been satisfied
@@ -732,7 +724,7 @@ public class RoleHistory {
         sortRecentNodeList(role);
       }
     }
-    // AA placement: now request a new node
+    // TODO: AA placement: now request a new node
 
     return outcome;
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
index 20f5802..4197c4f 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
@@ -21,17 +21,20 @@ package org.apache.slider.server.appmaster.state;
 import org.apache.slider.api.types.ComponentInformation;
 import org.apache.slider.providers.PlacementPolicy;
 import org.apache.slider.providers.ProviderRole;
+import org.apache.slider.server.appmaster.management.LongGauge;
 
 import java.io.Serializable;
 import java.util.Comparator;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-
 
 /**
- * Models the ongoing status of all nodes in  
- * Nothing here is synchronized: grab the whole instance to update.
+ * Models the ongoing status of all nodes in an application.
+ *
+ * These structures are shared across the {@link AppState} and {@link RoleHistory} structures,
+ * and must be designed for synchronous access. Atomic counters are preferred to anything which
+ * requires synchronization. Where synchronized access is good is that it allows for
+ * the whole instance to be locked, for updating multiple entries.
  */
 public final class RoleStatus implements Cloneable {
 
@@ -43,13 +46,19 @@ public final class RoleStatus implements Cloneable {
   private final int key;
   private final ProviderRole providerRole;
 
-  private int desired, actual, requested, releasing;
-  private int failed, startFailed;
-  private int started,  completed, totalRequested;
-  private final AtomicLong preempted = new AtomicLong(0);
-  private final AtomicLong nodeFailed = new AtomicLong(0);
-  private final AtomicLong failedRecently = new AtomicLong(0);
-  private final AtomicLong limitsExceeded = new AtomicLong(0);
+  private final LongGauge desired = new LongGauge();
+  private final LongGauge actual = new LongGauge();
+  private final LongGauge requested = new LongGauge();
+  private final LongGauge releasing = new LongGauge();
+  private final LongGauge failed = new LongGauge();
+  private final LongGauge startFailed = new LongGauge();
+  private final LongGauge started= new LongGauge();
+  private final LongGauge completed = new LongGauge();
+  private final LongGauge totalRequested = new LongGauge();
+  private final LongGauge preempted = new LongGauge(0);
+  private final LongGauge nodeFailed = new LongGauge(0);
+  private final LongGauge failedRecently = new LongGauge(0);
+  private final LongGauge limitsExceeded = new LongGauge(0);
 
   /** flag set to true if there is an outstanding anti-affine request */
   private final AtomicBoolean pendingAARequest = new AtomicBoolean(false);
@@ -125,63 +134,61 @@ public final class RoleStatus implements Cloneable {
     return !hasPlacementPolicy(PlacementPolicy.NO_DATA_LOCALITY);
   }
 
-  public synchronized int getDesired() {
-    return desired;
+  public long getDesired() {
+    return desired.get();
   }
 
-  public synchronized void setDesired(int desired) {
-    this.desired = desired;
+  public void setDesired(long desired) {
+    this.desired.set(desired);
   }
 
-  public synchronized int getActual() {
-    return actual;
+  public long getActual() {
+    return actual.get();
   }
 
-  public synchronized int incActual() {
-    return ++actual;
+  public long incActual() {
+    return actual.incrementAndGet();
   }
 
-  public synchronized int decActual() {
-    actual = Math.max(0, actual - 1);
-    return actual;
+  public long decActual() {
+    return actual.decToFloor(1);
   }
 
-  public synchronized int getRequested() {
-    return requested;
+  public long getRequested() {
+    return requested.get();
   }
 
-  public synchronized int incRequested() {
-    totalRequested++;
-    return ++requested;
+  public long incRequested() {
+    totalRequested.incrementAndGet();
+    return requested.incrementAndGet();
   }
 
-  public synchronized int cancel(int count) {
-    requested = Math.max(0, requested - count);
-    return requested;
+  
+  public long cancel(long count) {
+    return requested.decToFloor(count);
   }
   
-  public synchronized int decRequested() {
-    return cancel(1);
+  public void decRequested() {
+    cancel(1);
   }
 
-  public synchronized int getReleasing() {
-    return releasing;
+  public long getReleasing() {
+    return releasing.get();
   }
 
-  public synchronized int incReleasing() {
-    return ++releasing;
+  public long incReleasing() {
+    return releasing.incrementAndGet();
   }
 
-  public synchronized int decReleasing() {
-    releasing = Math.max(0, releasing - 1);
-    return releasing;
+  public long decReleasing() {
+    return releasing.decToFloor(1);
   }
 
-  public synchronized int getFailed() {
-    return failed;
+  public long getFailed() {
+    return failed.get();
   }
 
-  public synchronized long getFailedRecently() {
+  public long getFailedRecently() {
     return failedRecently.get();
   }
 
@@ -217,7 +224,7 @@ public final class RoleStatus implements Cloneable {
 
       case Node_failure:
         nodeFailed.incrementAndGet();
-        failed++;
+        failed.incrementAndGet();
         break;
 
       case Failed_limits_exceeded: // exceeded memory or CPU; app/configuration related
@@ -225,7 +232,7 @@ public final class RoleStatus implements Cloneable {
         // fall through
       case Failed: // application failure, possibly node related, possibly not
       default: // anything else (future-proofing)
-        failed++;
+        failed.incrementAndGet();
         failedRecently.incrementAndGet();
         //have a look to see if it short lived
         if (startupFailure) {
@@ -235,39 +242,39 @@ public final class RoleStatus implements Cloneable {
     }
   }
 
-  public synchronized int getStartFailed() {
-    return startFailed;
+  public long getStartFailed() {
+    return startFailed.get();
   }
 
   public synchronized void incStartFailed() {
-    startFailed++;
+    startFailed.getAndIncrement();
   }
 
   public synchronized String getFailureMessage() {
     return failureMessage;
   }
 
-  public synchronized int getCompleted() {
-    return completed;
+  public long getCompleted() {
+    return completed.get();
   }
 
   public synchronized void setCompleted(int completed) {
-    this.completed = completed;
+    this.completed.set(completed);
   }
 
-  public synchronized int incCompleted() {
-    return completed ++;
+  public long incCompleted() {
+    return completed.incrementAndGet();
   }
-  public synchronized int getStarted() {
-    return started;
+  public long getStarted() {
+    return started.get();
   }
 
   public synchronized void incStarted() {
-    started++;
+    started.incrementAndGet();
   }
 
-  public synchronized int getTotalRequested() {
-    return totalRequested;
+  public long getTotalRequested() {
+    return totalRequested.get();
   }
 
   public long getPreempted() {
@@ -284,13 +291,13 @@ public final class RoleStatus implements Cloneable {
    * @return the positive or negative number of roles to add/release.
    * 0 means "do nothing".
    */
-  public synchronized int getDelta() {
-    int inuse = getActualAndRequested();
+  public long getDelta() {
+    long inuse = getActualAndRequested();
     //don't know how to view these. Are they in-use or not?
-    int delta = desired - inuse;
+    long delta = desired.get() - inuse;
     if (delta < 0) {
       //if we are releasing, remove the number that are already released.
-      delta += releasing;
+      delta += releasing.get();
       //but never switch to a positive
       delta = Math.min(delta, 0);
     }
@@ -301,8 +308,8 @@ public final class RoleStatus implements Cloneable {
    * Get count of actual and requested containers
    * @return the size of the application when outstanding requests are included
    */
-  public synchronized int getActualAndRequested() {
-    return actual + requested;
+  public long getActualAndRequested() {
+    return actual.get() + requested.get();
   }
 
   @Override
@@ -357,15 +364,15 @@ public final class RoleStatus implements Cloneable {
     ComponentInformation info = new ComponentInformation();
     info.name = name;
     info.priority = getPriority();
-    info.desired = desired;
-    info.actual = actual;
-    info.requested = requested;
-    info.releasing = releasing;
-    info.failed = failed;
-    info.startFailed = startFailed;
+    info.desired = desired.intValue();
+    info.actual = actual.intValue();
+    info.requested = requested.intValue();
+    info.releasing = releasing.intValue();
+    info.failed = failed.intValue();
+    info.startFailed = startFailed.intValue();
     info.placementPolicy = getPlacementPolicy();
     info.failureMessage = failureMessage;
-    info.totalRequested = totalRequested;
+    info.totalRequested = totalRequested.intValue();
     info.failedRecently = failedRecently.intValue();
     info.nodeFailed = nodeFailed.intValue();
     info.preempted = preempted.intValue();

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/main/java/org/apache/slider/server/appmaster/state/SimpleReleaseSelector.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/SimpleReleaseSelector.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/SimpleReleaseSelector.java
index b7f0e05..b848096 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/SimpleReleaseSelector.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/SimpleReleaseSelector.java
@@ -27,8 +27,7 @@ public class SimpleReleaseSelector implements ContainerReleaseSelector {
 
   @Override
   public List<RoleInstance> sortCandidates(int roleId,
-      List<RoleInstance> candidates,
-      int minimumToSelect) {
+      List<RoleInstance> candidates) {
     return candidates;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java
index 0896e2b..65d8b39 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java
@@ -51,6 +51,7 @@ public class ContainerStatsBlock extends HtmlBlock {
 
   // Some functions that help transform the data into an object we can use to abstract presentation specifics
   protected static final Function<Entry<String,Integer>,Entry<TableContent,Integer>> stringIntPairFunc = toTableContentFunction();
+  protected static final Function<Entry<String,Long>,Entry<TableContent,Long>> stringLongPairFunc = toTableContentFunction();
   protected static final Function<Entry<String,String>,Entry<TableContent,String>> stringStringPairFunc = toTableContentFunction();
 
   private WebAppApi slider;
@@ -108,7 +109,7 @@ public class ContainerStatsBlock extends HtmlBlock {
       DIV<Hamlet> div = html.div("role-info ui-widget-content ui-corner-all");
 
       List<ClusterNode> nodesInRole =
-          new ArrayList<ClusterNode>(clusterNodesInRole.values());
+          new ArrayList<>(clusterNodesInRole.values());
 
       div.h2(BOLD, StringUtils.capitalize(name));
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/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 810affc..157870a 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
@@ -55,41 +55,6 @@ class TestMockAppStateAAPlacement extends BaseMockAppStateTest
   }*/
 
   /**
-   * Get the container request of an indexed entry. Includes some assertions for better diagnostics
-   * @param ops operation list
-   * @param index index in the list
-   * @return the request.
-   */
-  AMRMClient.ContainerRequest getRequest(List<AbstractRMOperation> ops, int index) {
-    assert index < ops.size()
-    def op = ops[index]
-    assert op instanceof ContainerRequestOperation
-    ((ContainerRequestOperation) op).request
-  }
-
-  /**
-   * Get the cancel request of an indexed entry. Includes some assertions for better diagnostics
-   * @param ops operation list
-   * @param index index in the list
-   * @return the request.
-   */
-  AMRMClient.ContainerRequest getCancel(List<AbstractRMOperation> ops, int index) {
-    assert index < ops.size()
-    def op = ops[index]
-    assert op instanceof CancelSingleRequest
-    ((CancelSingleRequest) op).request
-  }
-
-  /**
-   * Get the single request of a list of operations; includes the check for the size
-   * @param ops operations list of size 1
-   * @return the request within the first ContainerRequestOperation
-   */
-  public AMRMClient.ContainerRequest getSingleRequest(List<AbstractRMOperation> ops) {
-    assert 1 == ops.size()
-    getRequest(ops, 0)
-  }
-  /**
    * Get the single request of a list of operations; includes the check for the size
    * @param ops operations list of size 1
    * @return the request within the first operation
@@ -123,10 +88,6 @@ class TestMockAppStateAAPlacement extends BaseMockAppStateTest
 
     Container allocated = engine.allocateContainer(request)
 
-    // node is allocated wherever
-
-    def firstAllocation = allocated.nodeId
-
     // notify the container ane expect
     List<ContainerAssignment> assignments = [];
     List<AbstractRMOperation> operations = []
@@ -142,9 +103,10 @@ class TestMockAppStateAAPlacement extends BaseMockAppStateTest
     // we also expect a new allocation request to have been issued
 
     def req2 = getRequest(operations, 1)
-    // now the nodes should be a list
     Container allocated2 = engine.allocateContainer(req2)
 
+    // placement must be on a different host
+    assert allocated2.nodeId != allocated.nodeId
 
     ContainerAssignment assigned = assignments[0]
     Container container = assigned.container

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicHistory.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicHistory.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicHistory.groovy
index c62eb72..e57f341 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicHistory.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicHistory.groovy
@@ -20,7 +20,6 @@ package org.apache.slider.server.appmaster.model.appstate
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.yarn.api.records.ContainerId
 import org.apache.slider.api.ResourceKeys
 import org.apache.slider.core.conf.ConfTreeOperations
@@ -28,16 +27,14 @@ import org.apache.slider.core.exceptions.BadConfigException
 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.MockAppState
 import org.apache.slider.server.appmaster.model.mock.MockRoleHistory
 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.ContainerRequestOperation
 import org.apache.slider.server.appmaster.state.AppState
-import org.apache.slider.server.appmaster.state.AppStateBindingInfo
 import org.apache.slider.server.appmaster.state.NodeInstance
 import org.apache.slider.server.appmaster.state.RoleInstance
-import org.apache.slider.server.appmaster.state.SimpleReleaseSelector
+import org.apache.slider.server.appmaster.state.RoleStatus
 import org.junit.Test
 
 /**
@@ -191,21 +188,16 @@ class TestMockAppStateDynamicHistory extends BaseMockAppStateTest
   @Test(expected = BadConfigException.class)
   public void testRoleHistoryRoleAdditions() throws Throwable {
     MockRoleHistory roleHistory = new MockRoleHistory([])
-    roleHistory.addNewProviderRole(new ProviderRole("one", 1))
-    roleHistory.addNewProviderRole(new ProviderRole("two", 1))
+    roleHistory.addNewRole(new RoleStatus(new ProviderRole("one", 1)))
+    roleHistory.addNewRole(new RoleStatus(new ProviderRole("two", 1)))
     roleHistory.dump()
-    fail("should have raised an exception")
   }
-  
-  
+
   @Test(expected = BadConfigException.class)
   public void testRoleHistoryRoleStartupConflict() throws Throwable {
     MockRoleHistory roleHistory = new MockRoleHistory([
         new ProviderRole("one", 1), new ProviderRole("two", 1)
     ])
     roleHistory.dump()
-    fail("should have raised an exception")
   }
-  
-  
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.groovy
index 05b38ab..d0163d2 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.groovy
@@ -98,21 +98,16 @@ class TestMockAppStateDynamicRoles extends BaseMockAppStateTest
    * @param actions source list
    * @return found list
    */
-  List<ContainerRequestOperation> findAllocationsForRole(int role, 
+  Collection<ContainerRequestOperation> findAllocationsForRole(int role,
       List<AbstractRMOperation> actions) {
-    List <ContainerRequestOperation > results = []
-    actions.each { AbstractRMOperation  operation ->
-      if (operation instanceof ContainerRequestOperation) {
-        def req = (ContainerRequestOperation) operation;
-        def reqrole = ContainerPriority.extractRole(req.request.priority)
-        if (role == reqrole) {
-          results << req
-        }
-      }
+    def requests = actions.findAll {
+      it instanceof ContainerRequestOperation}.collect {it as ContainerRequestOperation}
+
+    requests.findAll {
+        role == ContainerPriority.extractRole(it.request.priority)
     }
-    return results
-  } 
-  
+  }
+
   @Test
   public void testStrictPlacementInitialRequest() throws Throwable {
     log.info("Initial engine state = $engine")
@@ -124,7 +119,6 @@ class TestMockAppStateDynamicRoles extends BaseMockAppStateTest
     assertRelaxLocalityFlag(ID5, null, true, actions)
   }
 
-
   @Test
   public void testPolicyPropagation() throws Throwable {
     assert !(appState.lookupRoleStatus(ROLE4).placementPolicy & PlacementPolicy.STRICT)
@@ -136,7 +130,6 @@ class TestMockAppStateDynamicRoles extends BaseMockAppStateTest
   public void testNodeFailureThresholdPropagation() throws Throwable {
     assert (appState.lookupRoleStatus(ROLE4).nodeFailureThreshold == 3)
     assert (appState.lookupRoleStatus(ROLE5).nodeFailureThreshold == 2)
-
   }
 
   @Test
@@ -156,7 +149,6 @@ class TestMockAppStateDynamicRoles extends BaseMockAppStateTest
     assert instanceA
     def hostname = RoleHistoryUtils.hostnameOf(instanceA.container)
 
-
     log.info("Allocated engine state = $engine")
     assert engine.containerCount() == 1
 
@@ -166,8 +158,7 @@ class TestMockAppStateDynamicRoles extends BaseMockAppStateTest
     role4.desired = 0
     appState.lookupRoleStatus(ROLE4).desired = 0
     def completionResults = []
-    def containersToRelease = []
-    instances = createStartAndStopNodes(completionResults)
+    createStartAndStopNodes(completionResults)
     assert engine.containerCount() == 0
     assert completionResults.size() == 1
 
@@ -198,19 +189,16 @@ class TestMockAppStateDynamicRoles extends BaseMockAppStateTest
     }
     assert instanceA
     def hostname = RoleHistoryUtils.hostnameOf(instanceA.container)
-    
-
 
     log.info("Allocated engine state = $engine")
     assert engine.containerCount() == 1
 
     assert role5.actual == 1
-    // shrinking cluster
 
+    // shrinking cluster
     role5.desired = 0
     def completionResults = []
-    def containersToRelease = []
-    instances = createStartAndStopNodes(completionResults)
+    createStartAndStopNodes(completionResults)
     assert engine.containerCount() == 0
     assert completionResults.size() == 1
     assert role5.actual == 0
@@ -223,16 +211,15 @@ class TestMockAppStateDynamicRoles extends BaseMockAppStateTest
     def nodes = cro.request.nodes
     assert nodes.size() == 1
     assert hostname == nodes[0]
-    
   }
 
   public void assertRelaxLocalityFlag(
-      int id,
+      int role,
       String expectedHost,
       boolean expectedRelaxFlag,
       List<AbstractRMOperation> actions) {
     def requests
-    requests = findAllocationsForRole(id, actions)
+    requests = findAllocationsForRole(role, actions)
     assert requests.size() == 1
     def req = requests[0]
     assert expectedRelaxFlag == req.request.relaxLocality

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.groovy
index 8ab63aa..5609682 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.groovy
@@ -33,6 +33,7 @@ import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
 import org.apache.slider.server.appmaster.model.mock.MockContainer
 import org.apache.slider.server.appmaster.model.mock.MockFactory
 import org.apache.slider.server.appmaster.model.mock.MockNodeId
+import org.apache.slider.server.appmaster.model.mock.MockRoleHistory
 import org.apache.slider.server.appmaster.state.*
 import org.junit.Before
 import org.junit.Test
@@ -60,7 +61,7 @@ class TestRoleHistoryContainerEvents extends BaseMockAppStateTest {
   String roleName = "test"
 
   List<NodeInstance> nodes = [age2Active2, age2Active0, age4Active1, age1Active4, age3Active0]
-  RoleHistory roleHistory = new RoleHistory(MockFactory.ROLES)
+  RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES)
 
   Resource resource
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryFindNodesForNewInstances.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryFindNodesForNewInstances.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryFindNodesForNewInstances.groovy
index c4768ec..63aa6d2 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryFindNodesForNewInstances.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryFindNodesForNewInstances.groovy
@@ -23,6 +23,7 @@ import groovy.util.logging.Slf4j
 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.MockRoleHistory
 import org.apache.slider.server.appmaster.state.ContainerOutcome
 import org.apache.slider.server.appmaster.state.NodeInstance
 import org.apache.slider.server.appmaster.state.RoleHistory
@@ -54,7 +55,7 @@ class TestRoleHistoryFindNodesForNewInstances extends BaseMockAppStateTest {
   NodeInstance empty = new NodeInstance("empty", MockFactory.ROLE_COUNT)
 
   List<NodeInstance> nodes = [age2Active2, age2Active0, age4Active1, age1Active4, age3Active0]
-  RoleHistory roleHistory = new RoleHistory(MockFactory.ROLES)
+  RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES)
 
   String roleName = "test"
   RoleStatus roleStat = new RoleStatus(new ProviderRole(roleName, 0))

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryNIComparators.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryNIComparators.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryNIComparators.groovy
index ee910e4..bcd8f9f 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryNIComparators.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryNIComparators.groovy
@@ -40,6 +40,7 @@ class TestRoleHistoryNIComparators extends BaseMockAppStateTest  {
   NodeInstance age1failing = nodeInstance(1001, 0, 0, 0)
 
   List<NodeInstance> nodes = [age2Active2, age4Active1, age1Active4, age3Active0]
+  List<NodeInstance> nodesPlusEmpty = [age2Active2, age4Active1, age1Active4, age3Active0, empty]
   List<NodeInstance> allnodes = [age6failing, age2Active2, age4Active1, age1Active4, age3Active0, age1failing]
 
   @Before
@@ -80,9 +81,8 @@ class TestRoleHistoryNIComparators extends BaseMockAppStateTest  {
 
   @Test
   public void testNewerThanNoRole() throws Throwable {
-    nodes << empty
-    Collections.sort(nodes, new NodeInstance.Preferred(0))
-    assertListEquals(nodes, [age4Active1, age3Active0, age2Active2, age1Active4, empty])
+    Collections.sort(nodesPlusEmpty, new NodeInstance.Preferred(0))
+    assertListEquals(nodesPlusEmpty, [age4Active1, age3Active0, age2Active2, age1Active4, empty])
   }
 
   @Test
@@ -94,9 +94,9 @@ class TestRoleHistoryNIComparators extends BaseMockAppStateTest  {
 
   @Test
   public void testMoreActiveThanEmpty() throws Throwable {
-    nodes << empty
-    Collections.sort(nodes, new NodeInstance.MoreActiveThan(0))
-    assertListEquals(nodes, [age1Active4, age2Active2, age4Active1, age3Active0, empty])
+
+    Collections.sort(nodesPlusEmpty, new NodeInstance.MoreActiveThan(0))
+    assertListEquals(nodesPlusEmpty, [age1Active4, age2Active2, age4Active1, age3Active0, empty])
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
index 254c0b6..72e4240 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.groovy
@@ -27,6 +27,7 @@ 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.MockRoleHistory
 import org.apache.slider.server.appmaster.state.NodeEntry
 import org.apache.slider.server.appmaster.state.NodeInstance
 import org.apache.slider.server.appmaster.state.RoleHistory
@@ -57,7 +58,7 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
 
   @Test
   public void testWriteReadEmpty() throws Throwable {
-    RoleHistory roleHistory = new RoleHistory(MockFactory.ROLES)
+    RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES)
     roleHistory.onStart(fs, historyPath)
     Path history = roleHistory.saveHistory(time++)
     assert fs.isFile(history)
@@ -67,7 +68,7 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
   
   @Test
   public void testWriteReadData() throws Throwable {
-    RoleHistory roleHistory = new RoleHistory(MockFactory.ROLES)
+    RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES)
     assert !roleHistory.onStart(fs, historyPath)
     String addr = "localhost"
     NodeInstance instance = roleHistory.getOrCreateNodeInstance(addr)
@@ -77,7 +78,7 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
     Path history = roleHistory.saveHistory(time++)
     assert fs.isFile(history)
     RoleHistoryWriter historyWriter = new RoleHistoryWriter();
-    RoleHistory rh2 = new RoleHistory(MockFactory.ROLES)
+    RoleHistory rh2 = new MockRoleHistory(MockFactory.ROLES)
 
 
     def loadedRoleHistory = historyWriter.read(fs, history)
@@ -92,7 +93,7 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
     
   @Test
   public void testWriteReadActiveData() throws Throwable {
-    RoleHistory roleHistory = new RoleHistory(MockFactory.ROLES)
+    RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES)
     roleHistory.onStart(fs, historyPath)
     String addr = "localhost"
     String addr2 = "rack1server5"
@@ -117,7 +118,7 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
     describe("Loaded")
     log.info("testWriteReadActiveData in $history")
     RoleHistoryWriter historyWriter = new RoleHistoryWriter();
-    RoleHistory rh2 = new RoleHistory(MockFactory.ROLES)
+    RoleHistory rh2 = new MockRoleHistory(MockFactory.ROLES)
     def loadedRoleHistory = historyWriter.read(fs, history)
     assert 3 == loadedRoleHistory.size()
     rh2.rebuild(loadedRoleHistory)
@@ -154,7 +155,7 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
 
   @Test
   public void testWriteThaw() throws Throwable {
-    RoleHistory roleHistory = new RoleHistory(MockFactory.ROLES)
+    RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES)
     assert !roleHistory.onStart(fs, historyPath)
     String addr = "localhost"
     NodeInstance instance = roleHistory.getOrCreateNodeInstance(addr)
@@ -164,7 +165,7 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
     Path history = roleHistory.saveHistory(time++)
     long savetime =roleHistory.saveTime;
     assert fs.isFile(history)
-    RoleHistory rh2 = new RoleHistory(MockFactory.ROLES)
+    RoleHistory rh2 = new MockRoleHistory(MockFactory.ROLES)
     assert rh2.onStart(fs, historyPath)
     NodeInstance ni2 = rh2.getExistingNodeInstance(addr)
     assert ni2 != null
@@ -211,7 +212,7 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
   @Test
   public void testSkipEmptyFileOnRead() throws Throwable {
     describe "verify that empty histories are skipped on read; old histories purged"
-    RoleHistory roleHistory = new RoleHistory(MockFactory.ROLES)
+    RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES)
     roleHistory.onStart(fs, historyPath)
     time = 0
     Path oldhistory = roleHistory.saveHistory(time++)
@@ -226,7 +227,7 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
     RoleHistoryWriter historyWriter = new RoleHistoryWriter();
     Path touched = touch(historyWriter, time++)
 
-    RoleHistory rh2 = new RoleHistory(MockFactory.ROLES)
+    RoleHistory rh2 = new MockRoleHistory(MockFactory.ROLES)
     assert rh2.onStart(fs, historyPath)
     NodeInstance ni2 = rh2.getExistingNodeInstance(addr)
     assert ni2 != null
@@ -240,7 +241,7 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
   @Test
   public void testSkipBrokenFileOnRead() throws Throwable {
     describe "verify that empty histories are skipped on read; old histories purged"
-    RoleHistory roleHistory = new RoleHistory(MockFactory.ROLES)
+    RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES)
     roleHistory.onStart(fs, historyPath)
     time = 0
     Path oldhistory = roleHistory.saveHistory(time++)
@@ -258,7 +259,7 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
     out.writeBytes("{broken:true}")
     out.close()
 
-    RoleHistory rh2 = new RoleHistory(MockFactory.ROLES)
+    RoleHistory rh2 = new MockRoleHistory(MockFactory.ROLES)
     describe("IGNORE STACK TRACE BELOW")
 
     assert rh2.onStart(fs, historyPath)
@@ -285,7 +286,7 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
 
     def loadedRoleHistory = writer.read(source)
     assert 4 == loadedRoleHistory.size()
-    RoleHistory roleHistory = new RoleHistory(MockFactory.ROLES)
+    RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES)
     assert 0 == roleHistory.rebuild(loadedRoleHistory)
   }
 
@@ -300,7 +301,7 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
 
     def loadedRoleHistory = writer.read(source)
     assert 6 == loadedRoleHistory.size()
-    RoleHistory roleHistory = new RoleHistory(MockFactory.ROLES)
+    RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES)
     assert 3 == roleHistory.rebuild(loadedRoleHistory)
   }
 
@@ -318,7 +319,7 @@ class TestRoleHistoryRW extends BaseMockAppStateTest {
     assert 4 == loadedRoleHistory.size()
     def expandedRoles = new ArrayList(MockFactory.ROLES)
     expandedRoles << PROVIDER_ROLE3
-    RoleHistory roleHistory = new RoleHistory(expandedRoles)
+    RoleHistory roleHistory = new MockRoleHistory(expandedRoles)
     assert 0 == roleHistory.rebuild(loadedRoleHistory)
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRWOrdering.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRWOrdering.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRWOrdering.groovy
index 0655531..8a0c1ca 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRWOrdering.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRWOrdering.groovy
@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path
 import org.apache.slider.common.SliderKeys
 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.MockRoleHistory
 import org.apache.slider.server.appmaster.state.NodeEntry
 import org.apache.slider.server.appmaster.state.NodeInstance
 import org.apache.slider.server.appmaster.state.RoleHistory
@@ -40,10 +41,10 @@ class TestRoleHistoryRWOrdering extends BaseMockAppStateTest {
 
   List<Path> paths = pathlist(
       [
-          "hdfs://localhost/history-0406c.json",
-          "hdfs://localhost/history-5fffa.json",
-          "hdfs://localhost/history-0001a.json",
-          "hdfs://localhost/history-0001f.json",
+        "hdfs://localhost/history-0406c.json",
+        "hdfs://localhost/history-5fffa.json",
+        "hdfs://localhost/history-0001a.json",
+        "hdfs://localhost/history-0001f.json",
       ]
   )
   Path h_0406c = paths[0]
@@ -52,9 +53,7 @@ class TestRoleHistoryRWOrdering extends BaseMockAppStateTest {
 
 
   List<Path> pathlist(List<String> pathnames) {
-    def result = []
-    pathnames.each { result << new Path(new URI(it as String)) }
-    result
+    pathnames.collect{ new Path(new URI(it as String)) }
   }
 
   @Override
@@ -85,7 +84,7 @@ class TestRoleHistoryRWOrdering extends BaseMockAppStateTest {
     describe "test that if multiple entries are written, the newest is picked up"
     long time = System.currentTimeMillis();
 
-    RoleHistory roleHistory = new RoleHistory(MockFactory.ROLES)
+    RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES)
     assert !roleHistory.onStart(fs, historyPath)
     String addr = "localhost"
     NodeInstance instance = roleHistory.getOrCreateNodeInstance(addr)
@@ -94,7 +93,7 @@ class TestRoleHistoryRWOrdering extends BaseMockAppStateTest {
 
     Path history1 = roleHistory.saveHistory(time++)
     Path history2 = roleHistory.saveHistory(time++)
-    Path history3 = roleHistory.saveHistory(time++)
+    Path history3 = roleHistory.saveHistory(time)
     
     //inject a later file with a different name
     sliderFileSystem.cat(new Path(historyPath, "file.json"), true, "hello, world")
@@ -137,10 +136,10 @@ class TestRoleHistoryRWOrdering extends BaseMockAppStateTest {
     RoleHistoryWriter.sortHistoryPaths(paths2)
     assertListEquals(paths2,
                      [
-                         paths[1],
-                         paths[0],
-                         paths[3],
-                         paths[2]
+                       paths[1],
+                       paths[0],
+                       paths[3],
+                       paths[2]
                      ])
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
index c6dcb07..693ea9f 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.groovy
@@ -28,6 +28,7 @@ import org.apache.slider.providers.ProviderRole
 import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
 import org.apache.slider.server.appmaster.model.mock.MockContainer
 import org.apache.slider.server.appmaster.model.mock.MockFactory
+import org.apache.slider.server.appmaster.model.mock.MockRoleHistory
 import org.apache.slider.server.appmaster.state.ContainerAllocationOutcome
 import org.apache.slider.server.appmaster.state.NodeEntry
 import org.apache.slider.server.appmaster.state.NodeInstance
@@ -55,7 +56,7 @@ class TestRoleHistoryRequestTracking extends BaseMockAppStateTest {
   NodeInstance empty = new NodeInstance("empty", MockFactory.ROLE_COUNT)
 
   List<NodeInstance> nodes = [age2Active2, age2Active0, age4Active1, age1Active4, age3Active0]
-  RoleHistory roleHistory = new RoleHistory(MockFactory.ROLES)
+  RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES)
   /** 1MB, 1 vcore*/
   Resource resource = Resource.newInstance(1, 1)
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
index 44d35be..cefba42 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.groovy
@@ -27,12 +27,15 @@ import org.apache.hadoop.yarn.api.records.ContainerId
 import org.apache.hadoop.yarn.api.records.ContainerState
 import org.apache.hadoop.yarn.api.records.ContainerStatus
 import org.apache.hadoop.yarn.api.records.NodeReport
+import org.apache.hadoop.yarn.client.api.AMRMClient
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.slider.common.tools.SliderFileSystem
 import org.apache.slider.common.tools.SliderUtils
 import org.apache.slider.core.conf.AggregateConf
 import org.apache.slider.core.main.LauncherExitCodes
 import org.apache.slider.server.appmaster.operations.AbstractRMOperation
+import org.apache.slider.server.appmaster.operations.CancelSingleRequest
+import org.apache.slider.server.appmaster.operations.ContainerRequestOperation
 import org.apache.slider.server.appmaster.state.AppState
 import org.apache.slider.server.appmaster.state.AppStateBindingInfo
 import org.apache.slider.server.appmaster.state.ContainerAssignment
@@ -57,15 +60,6 @@ abstract class BaseMockAppStateTest extends SliderTestBase implements MockRoles
   protected MockApplicationId applicationId;
   protected MockApplicationAttemptId applicationAttemptId;
 
-  @Override
-  void setup() {
-    super.setup()
-    YarnConfiguration conf = SliderUtils.createConfiguration()
-    fs = HadoopFS.get(new URI("file:///"), conf)
-    sliderFileSystem = new SliderFileSystem(fs, conf)
-    engine = createYarnEngine()
-  }
-
   /**
    * Override point: called in setup() to create the YARN engine; can
    * be changed for different sizes and options
@@ -75,13 +69,22 @@ abstract class BaseMockAppStateTest extends SliderTestBase implements MockRoles
     return new MockYarnEngine(64, 1)
   }
 
+
+  @Override
+  void setup() {
+    super.setup()
+    YarnConfiguration conf = SliderUtils.createConfiguration()
+    fs = HadoopFS.get(new URI("file:///"), conf)
+    sliderFileSystem = new SliderFileSystem(fs, conf)
+    engine = createYarnEngine()
+    initApp()
+  }
+
   /**
    * Initialize the application.
    * This uses the binding information supplied by {@link #buildBindingInfo()}.
    */
-  @Before
   void initApp(){
-
     String historyDirName = testName;
     YarnConfiguration conf = SliderUtils.createConfiguration()
     applicationId = new MockApplicationId(id: 1, clusterTimestamp: 0)
@@ -291,27 +294,27 @@ abstract class BaseMockAppStateTest extends SliderTestBase implements MockRoles
   /**
    * Process the RM operations and send <code>onContainersAllocated</code>
    * events to the app state
-   * @param ops
-   * @param released
-   * @return
+   * @param operationsIn list of incoming ops
+   * @param released released containers
+   * @return list of outbound operations
    */
   public List<RoleInstance> submitOperations(
-      List<AbstractRMOperation> ops,
-      List<ContainerId> released) {
-    List<Container> allocatedContainers = engine.execute(ops, released)
+      List<AbstractRMOperation> operationsIn,
+      List<ContainerId> released,
+      List<AbstractRMOperation> operationsOut = []) {
+    List<Container> allocatedContainers = engine.execute(operationsIn, released)
     List<ContainerAssignment> assignments = [];
-    List<AbstractRMOperation> operations = []
-    appState.onContainersAllocated(allocatedContainers, assignments, operations)
-    List<RoleInstance> instances = []
-    for (ContainerAssignment assigned : assignments) {
+    appState.onContainersAllocated(allocatedContainers, assignments, operationsOut)
+
+    assignments.collect {
+      ContainerAssignment assigned ->
       Container container = assigned.container
       RoleInstance ri = roleInstance(assigned)
-      instances << ri
       //tell the app it arrived
       log.debug("Start submitted ${ri.role} on ${container.id} ")
       appState.containerStartSubmitted(container, ri);
+      ri
     }
-    return instances
   }
 
   /**
@@ -334,13 +337,7 @@ abstract class BaseMockAppStateTest extends SliderTestBase implements MockRoles
   List<ContainerId> extractContainerIds(
       List<RoleInstance> instances,
       int role) {
-    List<ContainerId> cids = []
-    instances.each { RoleInstance instance ->
-      if (instance.roleId == role) {
-        cids << instance.id
-      }
-    }
-    return cids
+    instances.findAll { it.roleId == role }.collect { RoleInstance instance -> instance.id }
   }
 
   /**
@@ -363,4 +360,40 @@ abstract class BaseMockAppStateTest extends SliderTestBase implements MockRoles
   def recordAllFailed(int id, int count, List<NodeInstance> nodes) {
     nodes.each { NodeInstance node -> recordAsFailed(node, id, count)}
   }
+
+  /**
+   * Get the container request of an indexed entry. Includes some assertions for better diagnostics
+   * @param ops operation list
+   * @param index index in the list
+   * @return the request.
+   */
+  AMRMClient.ContainerRequest getRequest(List<AbstractRMOperation> ops, int index) {
+    assert index < ops.size()
+    def op = ops[index]
+    assert op instanceof ContainerRequestOperation
+    ((ContainerRequestOperation) op).request
+  }
+
+  /**
+   * Get the cancel request of an indexed entry. Includes some assertions for better diagnostics
+   * @param ops operation list
+   * @param index index in the list
+   * @return the request.
+   */
+  AMRMClient.ContainerRequest getCancel(List<AbstractRMOperation> ops, int index) {
+    assert index < ops.size()
+    def op = ops[index]
+    assert op instanceof CancelSingleRequest
+    ((CancelSingleRequest) op).request
+  }
+
+  /**
+   * Get the single request of a list of operations; includes the check for the size
+   * @param ops operations list of size 1
+   * @return the request within the first ContainerRequestOperation
+   */
+  public AMRMClient.ContainerRequest getSingleRequest(List<AbstractRMOperation> ops) {
+    assert 1 == ops.size()
+    getRequest(ops, 0)
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5b7f6dde/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockRoleHistory.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockRoleHistory.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockRoleHistory.groovy
index c521697..0a68afb 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockRoleHistory.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockRoleHistory.groovy
@@ -21,15 +21,21 @@ package org.apache.slider.server.appmaster.model.mock
 import org.apache.slider.core.exceptions.BadConfigException
 import org.apache.slider.providers.ProviderRole
 import org.apache.slider.server.appmaster.state.RoleHistory
+import org.apache.slider.server.appmaster.state.RoleStatus
 
 /**
  * subclass to enable access to some of the protected methods
  */
 class MockRoleHistory extends RoleHistory {
 
+  /**
+   * Take a list of provider roles and build the history from them, dynamically creating
+   * the role status entries on the way
+   * @param providerRoles provider role list
+   * @throws BadConfigException configuration problem with the role list
+   */
   MockRoleHistory(List<ProviderRole> providerRoles) throws BadConfigException {
-    super(providerRoles)
+    super(providerRoles.collect { new RoleStatus(it) })
   }
-  
-  
+
 }


[12/22] incubator-slider git commit: SLIDER-82 some minor IDE-suggested cleanup

Posted by st...@apache.org.
SLIDER-82 some minor IDE-suggested cleanup


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: 138912cce9a092a4babd054124cfe3babb12a303
Parents: 7f96416
Author: Steve Loughran <st...@apache.org>
Authored: Fri Nov 6 10:42:58 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Nov 6 12:06:11 2015 +0000

----------------------------------------------------------------------
 .../java/org/apache/slider/api/ClusterDescription.java  |  8 ++++----
 .../org/apache/slider/common/tools/SliderUtils.java     | 12 +++++-------
 2 files changed, 9 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/138912cc/slider-core/src/main/java/org/apache/slider/api/ClusterDescription.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/api/ClusterDescription.java b/slider-core/src/main/java/org/apache/slider/api/ClusterDescription.java
index 025bd32..7e3a9b4 100644
--- a/slider-core/src/main/java/org/apache/slider/api/ClusterDescription.java
+++ b/slider-core/src/main/java/org/apache/slider/api/ClusterDescription.java
@@ -166,14 +166,14 @@ public class ClusterDescription implements Cloneable {
    * the Slider AM and the application that it deploys
    */
   public Map<String, String> options =
-    new HashMap<String, String>();
+    new HashMap<>();
 
   /**
    * cluster information
    * This is only valid when querying the cluster status.
    */
   public Map<String, String> info =
-    new HashMap<String, String>();
+    new HashMap<>();
 
   /**
    * Statistics. This is only relevant when querying the cluster status
@@ -199,7 +199,7 @@ public class ClusterDescription implements Cloneable {
    * List of key-value pairs to add to a client config to set up the client
    */
   public Map<String, String> clientProperties =
-    new HashMap<String, String>();
+    new HashMap<>();
 
   /**
    * Status information
@@ -564,7 +564,7 @@ public class ClusterDescription implements Cloneable {
   public Map<String, String> getOrAddRole(String role) {
     Map<String, String> map = getRole(role);
     if (map == null) {
-      map = new HashMap<String, String>();
+      map = new HashMap<>();
     }
     roles.put(role, map);
     return map;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/138912cc/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
index 76668bf..5bf8622 100644
--- a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
+++ b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
@@ -746,10 +746,7 @@ public final class SliderUtils {
    *         through
    */
   public static boolean filter(String value, String filter) {
-    if (StringUtils.isEmpty(filter) || filter.equals(value)) {
-      return false;
-    }
-    return true;
+    return !(StringUtils.isEmpty(filter) || filter.equals(value));
   }
 
   /**
@@ -2028,10 +2025,10 @@ public final class SliderUtils {
       errorText.append("No native IO library. ");
     }
     try {
-      String path = Shell.getQualifiedBinPath("winutils.exe");
+      String path = Shell.getQualifiedBinPath(WINUTILS);
       log.debug("winutils is at {}", path);
     } catch (IOException e) {
-      errorText.append("No WINUTILS.EXE. ");
+      errorText.append("No " + WINUTILS);
       log.warn("No winutils: {}", e, e);
     }
     try {
@@ -2334,6 +2331,7 @@ public final class SliderUtils {
   public static String getClientConfigPath() {
     URL path = ConfigHelper.class.getClassLoader().getResource(
         SliderKeys.SLIDER_CLIENT_XML);
+    Preconditions.checkNotNull(path, "Failed to locate resource " + SliderKeys.SLIDER_CLIENT_XML);
     return path.toString();
   }
 
@@ -2483,7 +2481,7 @@ public final class SliderUtils {
 
   public static String requestToString(AMRMClient.ContainerRequest request) {
     Preconditions.checkArgument(request != null, "Null request");
-    StringBuffer buffer = new StringBuffer(request.toString());
+    StringBuilder buffer = new StringBuilder(request.toString());
     buffer.append("; ");
     buffer.append("relaxLocality=").append(request.getRelaxLocality()).append("; ");
     String labels = request.getNodeLabelExpression();


[03/22] incubator-slider git commit: SLIDER-82 minor test source cleanup

Posted by st...@apache.org.
SLIDER-82 minor test source cleanup


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: 9cda83be9e875ac649d99b711b41a2ae7d5e53ad
Parents: f2f2c76
Author: Steve Loughran <st...@apache.org>
Authored: Wed Nov 4 18:09:58 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Nov 5 13:19:03 2015 +0000

----------------------------------------------------------------------
 .../model/appstate/TestMockAppStateAppRestIntegration.groovy   | 5 -----
 .../model/appstate/TestMockAppStateDynamicRoles.groovy         | 6 ------
 .../appmaster/model/appstate/TestMockAppStateFlexing.groovy    | 3 +--
 .../model/appstate/TestMockAppStateRMOperations.groovy         | 4 +---
 .../model/appstate/TestMockAppStateRebuildOnAMRestart.groovy   | 3 +--
 .../model/appstate/TestMockContainerResourceAllocations.groovy | 5 -----
 6 files changed, 3 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/9cda83be/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 d36fdbc..6739623 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
@@ -41,11 +41,6 @@ import org.junit.Test
 @Slf4j
 class TestMockAppStateAppRestIntegration extends BaseMockAppStateTest implements MockRoles {
 
-  @Override
-  String getTestName() {
-    return "TestMockAppStateAppRestIntegration"
-  }
-
   @Test
   public void testCachedIntDocument() throws Throwable {
     ContentCache cache = new ContentCache()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/9cda83be/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.groovy
index e35f028..05b38ab 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.groovy
@@ -20,23 +20,17 @@ package org.apache.slider.server.appmaster.model.appstate
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.yarn.api.records.ContainerId
 import org.apache.slider.api.ResourceKeys
 import org.apache.slider.core.conf.AggregateConf
 import org.apache.slider.providers.PlacementPolicy
 import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
-import org.apache.slider.server.appmaster.model.mock.MockAppState
 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.operations.ContainerRequestOperation
-import org.apache.slider.server.appmaster.state.AppState
-import org.apache.slider.server.appmaster.state.AppStateBindingInfo
 import org.apache.slider.server.appmaster.state.ContainerPriority
 import org.apache.slider.server.appmaster.state.RoleHistoryUtils
 import org.apache.slider.server.appmaster.state.RoleInstance
-import org.apache.slider.server.appmaster.state.SimpleReleaseSelector
 import org.junit.Test
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/9cda83be/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexing.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexing.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexing.groovy
index 257092a..548842c 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexing.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexing.groovy
@@ -20,7 +20,6 @@ package org.apache.slider.server.appmaster.model.appstate
 
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.yarn.api.records.Container
-import org.apache.slider.api.ClusterDescription
 import org.apache.slider.core.exceptions.TriggerClusterTeardownException
 import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest
 import org.apache.slider.server.appmaster.model.mock.MockRoles
@@ -76,7 +75,7 @@ class TestMockAppStateFlexing extends BaseMockAppStateTest implements MockRoles
     ops = appState.reviewRequestAndReleaseNodes()
     assert ops.empty
 
-    RoleInstance ri2 = appState.innerOnNodeManagerContainerStarted(target.id)
+    appState.innerOnNodeManagerContainerStarted(target.id)
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/9cda83be/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRMOperations.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRMOperations.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRMOperations.groovy
index 9ac6fcf..ba7588a 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRMOperations.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRMOperations.groovy
@@ -110,7 +110,6 @@ class TestMockAppStateRMOperations extends BaseMockAppStateTest implements MockR
     // four outstanding
     assert role0.requested == 4
 
-
     // flex cluster to 3
     role0.desired = 3
     ops = appState.reviewRequestAndReleaseNodes()
@@ -226,8 +225,7 @@ class TestMockAppStateRMOperations extends BaseMockAppStateTest implements MockR
   public void testFlexUpNoSpace() throws Throwable {
     // engine only has two nodes, so > 2 will be outstanding
     engine = new MockYarnEngine(1, 2)
-    List<AbstractRMOperation> ops
-    // role: desired = 2, requested = 1, actual=1 
+    // role: desired = 2, requested = 1, actual=1
     def role0 = role0Status
     role0.desired = 4
     createAndSubmitNodes()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/9cda83be/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRebuildOnAMRestart.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRebuildOnAMRestart.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRebuildOnAMRestart.groovy
index 02052c2..59cc2c8 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRebuildOnAMRestart.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRebuildOnAMRestart.groovy
@@ -36,8 +36,7 @@ import org.junit.Test
  */
 @CompileStatic
 @Slf4j
-class TestMockAppStateRebuildOnAMRestart extends BaseMockAppStateTest
-    implements MockRoles {
+class TestMockAppStateRebuildOnAMRestart extends BaseMockAppStateTest implements MockRoles {
 
   @Override
   String getTestName() {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/9cda83be/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.groovy
index ad607cf..93cce95 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.groovy
@@ -38,11 +38,6 @@ import org.junit.Test
 @Slf4j
 class TestMockContainerResourceAllocations extends BaseMockAppStateTest {
 
-  @Override
-  String getTestName() {
-    "TestMockContainerResourceAllocations"
-  }
-
   @Test
   public void testNormalAllocations() throws Throwable {
     ConfTree clusterSpec = factory.newConfTree(1, 0, 0)