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 2016/02/29 22:40:33 UTC

incubator-slider git commit: SLIDER-1095 Antiaffinity test AASleepIT failing —says no. of allocated containers too low

Repository: incubator-slider
Updated Branches:
  refs/heads/develop dffff103e -> 3fcba9280


SLIDER-1095 Antiaffinity test AASleepIT failing —says no. of allocated containers too low


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

Branch: refs/heads/develop
Commit: 3fcba9280c2fd984bb8c3cefd69a3d59562604ec
Parents: dffff10
Author: Steve Loughran <st...@apache.org>
Authored: Mon Feb 29 21:39:57 2016 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Feb 29 21:39:57 2016 +0000

----------------------------------------------------------------------
 .../apache/slider/api/ClusterDescription.java   |   4 +
 .../apache/slider/test/SliderTestUtils.groovy   |   7 +-
 .../apache/slider/test/TestAssertions.groovy    |  14 +
 .../src/test/resources/json/cluster.json        | 266 +++++++++++++++++++
 .../funtest/framework/CommandTestBase.groovy    |   3 +-
 .../slider/funtest/lifecycle/AASleepIT.groovy   |  10 +-
 6 files changed, 294 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3fcba928/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 8358491..f8e5e7c 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
@@ -40,6 +40,7 @@ import org.slf4j.LoggerFactory;
 
 import java.io.DataOutputStream;
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -366,6 +367,9 @@ public class ClusterDescription implements Cloneable {
      */
     public static ClusterDescription fromStream(InputStream is)
             throws IOException, JsonParseException, JsonMappingException {
+        if (is==null) {
+          throw new FileNotFoundException("Empty Stream");
+        }
         ObjectMapper mapper = new ObjectMapper();
         try {
             return mapper.readValue(is, ClusterDescription.class);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3fcba928/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
index 0a3b040..239e660 100644
--- a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
@@ -47,6 +47,7 @@ import org.apache.slider.api.ClusterDescription
 import org.apache.slider.api.ClusterNode
 import org.apache.slider.api.RoleKeys
 import org.apache.slider.api.StateValues
+import org.apache.slider.api.StatusKeys
 import org.apache.slider.client.SliderClient
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.tools.Duration
@@ -745,7 +746,7 @@ class SliderTestUtils extends Assert {
     log.info("Asserting component $component expected count $expected}",)
     int actual = extractLiveContainerCount(clusterDescription, component)
     if (expected != actual) {
-      log.warn("$component actual=$actual, expected $expected in \n$clusterDescription")
+      log.warn("$component actual=$actual, expected $expected in \n$clusterDescription\n")
     }
     assert expected == actual
   }
@@ -759,8 +760,8 @@ class SliderTestUtils extends Assert {
   public static int extractLiveContainerCount(
       ClusterDescription clusterDescription,
       String component) {
-    def instances = clusterDescription?.instances?.get(component)
-    int actual = instances != null ? instances.size() : 0
+    def stats = clusterDescription?.statistics?.get(component)
+    int actual = stats != null ? stats.get(StatusKeys.STATISTICS_CONTAINERS_LIVE, 0) : 0
     return actual
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3fcba928/slider-core/src/test/groovy/org/apache/slider/test/TestAssertions.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/test/TestAssertions.groovy b/slider-core/src/test/groovy/org/apache/slider/test/TestAssertions.groovy
index abfbe65..200d284 100644
--- a/slider-core/src/test/groovy/org/apache/slider/test/TestAssertions.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/test/TestAssertions.groovy
@@ -19,12 +19,16 @@
 package org.apache.slider.test
 
 import org.apache.slider.api.ClusterDescription
+import org.apache.slider.api.StatusKeys
 import org.junit.Test
 
 /**
  * Test for some of the command test base operations
  */
 class TestAssertions {
+
+  public static final String CLUSTER_JSON = "json/cluster.json"
+
   @Test
   public void testNoInstances() throws Throwable {
     ClusterDescription clusterDescription = new ClusterDescription();
@@ -38,4 +42,14 @@ class TestAssertions {
     SliderTestUtils.assertContainersLive(clusterDescription, "example", 0);
   }
 
+  @Test
+  public void testLiveInstances() throws Throwable {
+    def stream = getClass().getClassLoader().getResourceAsStream(CLUSTER_JSON)
+    assert stream != null, "could not load $CLUSTER_JSON"
+    ClusterDescription liveCD = ClusterDescription.fromStream(stream)
+    assert liveCD != null
+    SliderTestUtils.assertContainersLive(liveCD, "SLEEP_LONG", 4)
+    assert 1 == liveCD.statistics["SLEEP_LONG"][StatusKeys.STATISTICS_CONTAINERS_ANTI_AFFINE_PENDING]
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3fcba928/slider-core/src/test/resources/json/cluster.json
----------------------------------------------------------------------
diff --git a/slider-core/src/test/resources/json/cluster.json b/slider-core/src/test/resources/json/cluster.json
new file mode 100644
index 0000000..3251b42
--- /dev/null
+++ b/slider-core/src/test/resources/json/cluster.json
@@ -0,0 +1,266 @@
+{
+  "version": "1.0",
+  "name": "test-aa-sleep",
+  "type": "agent",
+  "state": 3,
+  "createTime": 1455829237388,
+  "updateTime": 1455829240086,
+  "originConfigurationPath": "hdfs://slider-3.cluster:8020/user/qe/.slider/cluster/test-aa-sleep/snapshot",
+  "generatedConfigurationPath": "hdfs://slider-3.cluster:8020/user/qe/.slider/cluster/test-aa-sleep/generated",
+  "dataPath": "hdfs://slider-3.cluster:8020/user/qe/.slider/cluster/test-aa-sleep/database",
+  "options": {
+    "internal.provider.name": "agent",
+    "internal.container.failure.shortlife": "60000",
+    "internal.container.failure.threshold": "5",
+    "site.global.security_enabled": "true",
+    "zookeeper.quorum": "slider-3.cluster:2181,slider-2.cluster:2181,slider-1.cluster:2181",
+    "slider.cluster.directory.permissions": "0770",
+    "internal.generated.conf.path": "hdfs://slider-3.cluster:8020/user/qe/.slider/cluster/test-aa-sleep/generated",
+    "env.MALLOC_ARENA_MAX": "4",
+    "internal.tmp.dir": "hdfs://slider-3.cluster:8020/user/qe/.slider/cluster/test-aa-sleep/tmp",
+    "slider.data.directory.permissions": "0770",
+    "internal.am.tmp.dir": "hdfs://slider-3.cluster:8020/user/qe/.slider/cluster/test-aa-sleep/tmp/appmaster",
+    "internal.snapshot.conf.path": "hdfs://slider-3.cluster:8020/user/qe/.slider/cluster/test-aa-sleep/snapshot",
+    "application.def": "hdfs://slider-3.cluster:8020/user/qe/.slider/cluster/test-aa-sleep/appdef/appPkg.zip",
+    "zookeeper.hosts": "slider-3.cluster,slider-2.cluster,slider-1.cluster",
+    "zookeeper.path": "/services/slider/users/qe/test-aa-sleep",
+    "internal.data.dir.path": "hdfs://slider-3.cluster:8020/user/qe/.slider/cluster/test-aa-sleep/database",
+    "internal.addons.dir.path": "hdfs://slider-3.cluster:8020/user/qe/.slider/cluster/test-aa-sleep/tmp/addons",
+    "internal.application.image.path": null,
+    "internal.appdef.dir.path": "hdfs://slider-3.cluster:8020/user/qe/.slider/cluster/test-aa-sleep/tmp/appdef",
+    "site.fs.defaultFS": "hdfs://slider-3.cluster:8020",
+    "internal.application.home": null,
+    "site.dfs.namenode.kerberos.principal": "nn/_HOST@EXAMPLE.COM",
+    "site.fs.default.name": "hdfs://slider-3.cluster:8020",
+    "application.name": "test-aa-sleep"
+  },
+  "info": {
+    "yarn.vcores": "1",
+    "info.am.app.id": "application_1455824487784_0020",
+    "yarn.memory": "10240",
+    "info.am.web.url": "http://slider-3.cluster:1025/",
+    "info.am.rpc.port": "1024",
+    "info.am.hostname": "slider-3.cluster",
+    "info.am.web.port": "1025",
+    "info.am.container.id": "container_e01_1455824487784_0020_01_000001",
+    "info.am.attempt.id": "appattempt_1455824487784_0020_000001",
+    "live.time": "18 Feb 2016 21:00:37 GMT",
+    "live.time.millis": "1455829237388",
+    "create.time": "18 Feb 2016 21:00:37 GMT",
+    "create.time.millis": "1455829237388",
+    "containers.at.am-restart": "0",
+    "status.time": "18 Feb 2016 21:02:38 GMT",
+    "status.time.millis": "1455829358648",
+    "info.am.agent.status.url": "https://slider-3.cluster:46914/",
+    "info.am.agent.ops.url": "https://slider-3.cluster:39678/",
+    "info.am.agent.ops.port": "39678",
+    "info.am.agent.status.port": "46914"
+  },
+  "statistics": {
+    "SLEEP_100": {
+      "containers.failed.preempted": 0,
+      "containers.completed": 0,
+      "containers.failed": 0,
+      "containers.requested": 1,
+      "containers.desired": 1,
+      "containers.start.failed": 0,
+      "containers.failed.node": 0,
+      "containers.start.started": 0,
+      "containers.active.requests": 0,
+      "containers.failed.recently": 0,
+      "containers.live": 1,
+      "containers.anti-affine.pending": 0
+    },
+    "SLEEP_LONG": {
+      "containers.failed.preempted": 0,
+      "containers.completed": 0,
+      "containers.failed": 0,
+      "containers.requested": 4,
+      "containers.desired": 5,
+      "containers.start.failed": 0,
+      "containers.failed.node": 0,
+      "containers.start.started": 0,
+      "containers.active.requests": 0,
+      "containers.failed.recently": 0,
+      "containers.live": 4,
+      "containers.anti-affine.pending": 1
+    },
+    "slider-appmaster": {
+      "containers.completed": 0,
+      "containers.failed": 0,
+      "containers.unknown.completed": 0,
+      "containers.surplus": 0,
+      "containers.start.failed": 0,
+      "containers.start.started": 5,
+      "containers.live": 6
+    }
+  },
+  "instances": {
+    "SLEEP_100": [
+      "container_e01_1455824487784_0020_01_000003"
+    ],
+    "SLEEP_LONG": [
+      "container_e01_1455824487784_0020_01_000005",
+      "container_e01_1455824487784_0020_01_000004",
+      "container_e01_1455824487784_0020_01_000006",
+      "container_e01_1455824487784_0020_01_000002"
+    ],
+    "slider-appmaster": [
+      "container_e01_1455824487784_0020_01_000001"
+    ]
+  },
+  "roles": {
+    "slider-appmaster": {
+      "yarn.vcores": "1",
+      "yarn.memory": "256",
+      "role.releasing.instances": "0",
+      "role.failed.node.instances": "0",
+      "role.requested.instances": "0",
+      "role.actual.instances": "1",
+      "role.failed.recently.instances": "0",
+      "role.failed.starting.instances": "0",
+      "yarn.component.instances": "1",
+      "slider.keytab.principal.name": "qe@EXAMPLE.COM",
+      "role.failed.preempted.instances": "0",
+      "role.failed.instances": "0"
+    }
+  },
+  "clientProperties": {},
+  "status": {
+    "live": {
+      "SLEEP_100": {
+        "container_e01_1455824487784_0020_01_000003": {
+          "name": "container_e01_1455824487784_0020_01_000003",
+          "role": "SLEEP_100",
+          "roleId": 1,
+          "createTime": 1455829241537,
+          "startTime": 1455829241657,
+          "released": false,
+          "host": "slider-3.cluster",
+          "hostUrl": "http://slider-3.cluster:8042",
+          "state": 3,
+          "exitCode": 0,
+          "command": "python ./infra/agent/slider-agent/agent/main.py --label container_e01_1455824487784_0020_01_000003___SLEEP_100 --zk-quorum slider-3.cluster:2181,slider-2.cluster:2181,slider-1.cluster:2181 --zk-reg-path /registry/users/qe/services/org-apache-slider/test-aa-sleep > <LOG_DIR>/slider-agent.out 2>&1 ; ",
+          "environment": [
+            "LANGUAGE=\"en_US.UTF-8\"",
+            "PYTHONPATH=\"./infra/agent/slider-agent/\"",
+            "AGENT_LOG_ROOT=\"<LOG_DIR>\"",
+            "SLIDER_PASSPHRASE=\"ucaEi2Qyp3dRIz96wbSwU1SMv2SOesoFCAFYJAtGnQ4XqU32t0\"",
+            "LC_ALL=\"en_US.UTF-8\"",
+            "AGENT_WORK_ROOT=\"$PWD\"",
+            "LANG=\"en_US.UTF-8\""
+          ]
+        }
+      },
+      "SLEEP_LONG": {
+        "container_e01_1455824487784_0020_01_000006": {
+          "name": "container_e01_1455824487784_0020_01_000006",
+          "role": "SLEEP_LONG",
+          "roleId": 3,
+          "createTime": 1455829247516,
+          "startTime": 1455829247567,
+          "released": false,
+          "host": "slider-5.cluster",
+          "hostUrl": "http://slider-5.cluster:8042",
+          "state": 3,
+          "exitCode": 0,
+          "command": "python ./infra/agent/slider-agent/agent/main.py --label container_e01_1455824487784_0020_01_000006___SLEEP_LONG --zk-quorum slider-3.cluster:2181,slider-2.cluster:2181,slider-1.cluster:2181 --zk-reg-path /registry/users/qe/services/org-apache-slider/test-aa-sleep > <LOG_DIR>/slider-agent.out 2>&1 ; ",
+          "environment": [
+            "LANGUAGE=\"en_US.UTF-8\"",
+            "PYTHONPATH=\"./infra/agent/slider-agent/\"",
+            "AGENT_LOG_ROOT=\"<LOG_DIR>\"",
+            "SLIDER_PASSPHRASE=\"ucaEi2Qyp3dRIz96wbSwU1SMv2SOesoFCAFYJAtGnQ4XqU32t0\"",
+            "LC_ALL=\"en_US.UTF-8\"",
+            "AGENT_WORK_ROOT=\"$PWD\"",
+            "LANG=\"en_US.UTF-8\""
+          ]
+        },
+        "container_e01_1455824487784_0020_01_000004": {
+          "name": "container_e01_1455824487784_0020_01_000004",
+          "role": "SLEEP_LONG",
+          "roleId": 3,
+          "createTime": 1455829243488,
+          "startTime": 1455829243538,
+          "released": false,
+          "host": "slider-1.cluster",
+          "hostUrl": "http://slider-1.cluster:8042",
+          "state": 3,
+          "exitCode": 0,
+          "command": "python ./infra/agent/slider-agent/agent/main.py --label container_e01_1455824487784_0020_01_000004___SLEEP_LONG --zk-quorum slider-3.cluster:2181,slider-2.cluster:2181,slider-1.cluster:2181 --zk-reg-path /registry/users/qe/services/org-apache-slider/test-aa-sleep > <LOG_DIR>/slider-agent.out 2>&1 ; ",
+          "environment": [
+            "LANGUAGE=\"en_US.UTF-8\"",
+            "PYTHONPATH=\"./infra/agent/slider-agent/\"",
+            "AGENT_LOG_ROOT=\"<LOG_DIR>\"",
+            "SLIDER_PASSPHRASE=\"ucaEi2Qyp3dRIz96wbSwU1SMv2SOesoFCAFYJAtGnQ4XqU32t0\"",
+            "LC_ALL=\"en_US.UTF-8\"",
+            "AGENT_WORK_ROOT=\"$PWD\"",
+            "LANG=\"en_US.UTF-8\""
+          ]
+        },
+        "container_e01_1455824487784_0020_01_000005": {
+          "name": "container_e01_1455824487784_0020_01_000005",
+          "role": "SLEEP_LONG",
+          "roleId": 3,
+          "createTime": 1455829245506,
+          "startTime": 1455829245565,
+          "released": false,
+          "host": "slider-2.cluster",
+          "hostUrl": "http://slider-2.cluster:8042",
+          "state": 3,
+          "exitCode": 0,
+          "command": "python ./infra/agent/slider-agent/agent/main.py --label container_e01_1455824487784_0020_01_000005___SLEEP_LONG --zk-quorum slider-3.cluster:2181,slider-2.cluster:2181,slider-1.cluster:2181 --zk-reg-path /registry/users/qe/services/org-apache-slider/test-aa-sleep > <LOG_DIR>/slider-agent.out 2>&1 ; ",
+          "environment": [
+            "LANGUAGE=\"en_US.UTF-8\"",
+            "PYTHONPATH=\"./infra/agent/slider-agent/\"",
+            "AGENT_LOG_ROOT=\"<LOG_DIR>\"",
+            "SLIDER_PASSPHRASE=\"ucaEi2Qyp3dRIz96wbSwU1SMv2SOesoFCAFYJAtGnQ4XqU32t0\"",
+            "LC_ALL=\"en_US.UTF-8\"",
+            "AGENT_WORK_ROOT=\"$PWD\"",
+            "LANG=\"en_US.UTF-8\""
+          ]
+        },
+        "container_e01_1455824487784_0020_01_000002": {
+          "name": "container_e01_1455824487784_0020_01_000002",
+          "role": "SLEEP_LONG",
+          "roleId": 3,
+          "createTime": 1455829241551,
+          "startTime": 1455829241817,
+          "released": false,
+          "host": "slider-3.cluster",
+          "hostUrl": "http://slider-3.cluster:8042",
+          "state": 3,
+          "exitCode": 0,
+          "command": "python ./infra/agent/slider-agent/agent/main.py --label container_e01_1455824487784_0020_01_000002___SLEEP_LONG --zk-quorum slider-3.cluster:2181,slider-2.cluster:2181,slider-1.cluster:2181 --zk-reg-path /registry/users/qe/services/org-apache-slider/test-aa-sleep > <LOG_DIR>/slider-agent.out 2>&1 ; ",
+          "environment": [
+            "LANGUAGE=\"en_US.UTF-8\"",
+            "PYTHONPATH=\"./infra/agent/slider-agent/\"",
+            "AGENT_LOG_ROOT=\"<LOG_DIR>\"",
+            "SLIDER_PASSPHRASE=\"ucaEi2Qyp3dRIz96wbSwU1SMv2SOesoFCAFYJAtGnQ4XqU32t0\"",
+            "LC_ALL=\"en_US.UTF-8\"",
+            "AGENT_WORK_ROOT=\"$PWD\"",
+            "LANG=\"en_US.UTF-8\""
+          ]
+        }
+      },
+      "slider-appmaster": {
+        "container_e01_1455824487784_0020_01_000001": {
+          "name": "container_e01_1455824487784_0020_01_000001",
+          "role": "slider-appmaster",
+          "roleId": 0,
+          "createTime": 1455829237437,
+          "startTime": 1455829237437,
+          "released": false,
+          "host": "slider-3.cluster",
+          "hostUrl": "http://slider-3.cluster:1025",
+          "state": 3,
+          "exitCode": 0
+        }
+      }
+    }
+  },
+  "liveness": {
+    "allRequestsSatisfied": false,
+    "requestsOutstanding": 1,
+    "activeRequests": 0
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3fcba928/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
index 82589aa..b3bba74 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
@@ -1340,8 +1340,7 @@ abstract class CommandTestBase extends SliderTestUtils {
     int expectedCount = args['live'].toInteger();
     ClusterDescription cd = execStatus(application)
     def actual = extractLiveContainerCount(cd, component)
-    log.debug(
-        "live $component count = $actual; expected=$expectedCount")
+    log.debug("live $component count = $actual; expected=$expectedCount")
     return Outcome.fromBool(actual >= expectedCount)
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3fcba928/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AASleepIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AASleepIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AASleepIT.groovy
index faf9a25..f070ce3 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AASleepIT.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AASleepIT.groovy
@@ -24,6 +24,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.slider.api.ClusterDescription
 import org.apache.slider.api.ResourceKeys
 import org.apache.slider.api.RoleKeys
+import org.apache.slider.api.StatusKeys
 import org.apache.slider.api.types.NodeEntryInformation
 import org.apache.slider.api.types.NodeInformation
 import org.apache.slider.api.types.NodeInformationList
@@ -97,7 +98,8 @@ public class AASleepIT extends AgentCommandTestBase
       TEST_RESOURCE,
       ResourcePaths.SLEEP_APPCONFIG,
       [
-        ARG_RES_COMP_OPT, SLEEP_LONG, ResourceKeys.COMPONENT_INSTANCES, Integer.toString( desired),
+        ARG_RES_COMP_OPT, SLEEP_100, ResourceKeys.COMPONENT_INSTANCES, "0",
+        ARG_RES_COMP_OPT, SLEEP_LONG, ResourceKeys.COMPONENT_INSTANCES, Integer.toString(desired),
         ARG_RES_COMP_OPT, SLEEP_LONG, ResourceKeys.COMPONENT_PRIORITY, SLEEP_LONG_PRIORITY_S
       ],
       launchReportFile)
@@ -109,8 +111,6 @@ public class AASleepIT extends AgentCommandTestBase
     status(0, NAME)
 
     def expected = buildExpectedCount(desired)
-    expectLiveContainerCountReached(NAME, SLEEP_100, expected,
-        CONTAINER_LAUNCH_TIMEOUT)
 
     operations(NAME, loadAppReport(launchReportFile), desired, expected, healthyNodes)
 
@@ -145,7 +145,7 @@ public class AASleepIT extends AgentCommandTestBase
     // now here await for the cluster size to grow: if it does, there's a problem
     // spin for a while and fail if the number ever goes above it.
     ClusterDescription cd = null
-    5.times {
+    (desired * 5).times {
       cd = assertContainersLive(NAME, SLEEP_LONG, expected)
       sleep(1000 * 10)
     }
@@ -153,7 +153,7 @@ public class AASleepIT extends AgentCommandTestBase
     // here cluster is still 1 below expected
     def role = cd.getRole(SLEEP_LONG)
     assert "1" == role.get(RoleKeys.ROLE_PENDING_AA_INSTANCES)
-
+    assert 1 == cd.statistics[SLEEP_LONG][StatusKeys.STATISTICS_CONTAINERS_ANTI_AFFINE_PENDING]
     // look through the nodes
     def currentNodes = listNodes(name)
     // assert that there is no entry of the sleep long priority on any node