You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bi...@apache.org on 2023/06/20 23:18:17 UTC

[storm] branch master updated: [STORM-3916] Add Round Robin Scheduling strategy with optional node limit (#3540)

This is an automated email from the ASF dual-hosted git repository.

bipinprasad pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
     new 6a1049800 [STORM-3916] Add Round Robin Scheduling strategy with optional node limit (#3540)
6a1049800 is described below

commit 6a10498003ed9c759a9004aeda5f224ba58783c8
Author: Bipin Prasad <bi...@apache.org>
AuthorDate: Tue Jun 20 16:18:11 2023 -0700

    [STORM-3916] Add Round Robin Scheduling strategy with optional node limit (#3540)
    
    * [STORM-3916] Add Round Robin Scheduling stretgy with optional node limit.
    
    * [STORM-3916] Remove unused method isSortNodesForEachExecutor()
    
    * [STORM-3916] Add more javadoc explanation to config variable TOPOLOGY_ISOLATED_MACHINES.
---
 docs/Resource_Aware_Scheduler_overview.md          |    1 +
 storm-client/src/jvm/org/apache/storm/Config.java  |   14 +-
 .../storm/messaging/NettyIntegrationTest.java      |    8 +-
 .../scheduling/BaseResourceAwareStrategy.java      |   29 +-
 .../scheduling/ConstraintSolverConfig.java         |    1 -
 .../RoundRobinResourceAwareStrategy.java           |  187 +++
 .../test/java/org/apache/storm/TestRebalance.java  |  133 +-
 .../org/apache/storm/daemon/nimbus/NimbusTest.java |   25 +-
 .../blacklist/TestBlacklistScheduler.java          |   88 +-
 .../resource/TestResourceAwareScheduler.java       |  839 +++++------
 .../apache/storm/scheduler/resource/TestUser.java  |   63 +-
 .../TestUtilsForResourceAwareScheduler.java        |   68 +-
 .../eviction/TestDefaultEvictionStrategy.java      |  421 +++---
 .../TestFIFOSchedulingPriorityStrategy.java        |  117 +-
 ...ricResourceAwareSchedulingPriorityStrategy.java |  200 +--
 .../scheduling/TestBackwardCompatibility.java      |  435 ------
 .../TestDefaultResourceAwareStrategy.java          | 1503 ++++++++++----------
 .../TestGenericResourceAwareStrategy.java          |  974 +++++++------
 .../strategies/scheduling/TestLargeCluster.java    |  119 +-
 .../TestRoundRobinNodeSorterHostIsolation.java     |  265 ++++
 .../TestRoundRobinNodeSorterHostProximity.java     |  944 ++++++++++++
 21 files changed, 3821 insertions(+), 2613 deletions(-)

diff --git a/docs/Resource_Aware_Scheduler_overview.md b/docs/Resource_Aware_Scheduler_overview.md
index 4d03f88c5..7fb0e31e2 100644
--- a/docs/Resource_Aware_Scheduler_overview.md
+++ b/docs/Resource_Aware_Scheduler_overview.md
@@ -377,6 +377,7 @@ The metrics with -1:__system are generally metrics for the entire worker.  In th
 The Memory usage is similar but look at the usedBytes.  offHeap is 64621728 or about 62MB, and onHeap is 83857888 or about 80MB, but you should know what you set your heap to in each of your workers already.  How do you divide this up per bolt/spout?  That is a bit harder and may require some trial and error from your end.
 
 <div id='Enhancements-on-original-DefaultResourceAwareStrategy'/>
+
 ## Enhancements on original DefaultResourceAwareStrategy
 
 The default resource aware scheduling strategy as described in the paper above has two main scheduling phases:
diff --git a/storm-client/src/jvm/org/apache/storm/Config.java b/storm-client/src/jvm/org/apache/storm/Config.java
index 297498513..60135bc7c 100644
--- a/storm-client/src/jvm/org/apache/storm/Config.java
+++ b/storm-client/src/jvm/org/apache/storm/Config.java
@@ -859,9 +859,17 @@ public class Config extends HashMap<String, Object> {
     @IsString
     public static final String STORM_DO_AS_USER = "storm.doAsUser";
     /**
-     * The number of machines that should be used by this topology to isolate it from all others. Set storm.scheduler to
-     * org.apache.storm.scheduler.multitenant.MultitenantScheduler
-     */
+     * The maximum number of machines that should be used by this topology. This configuration can
+     * be used to isolate topologies from each other. See {@link org.apache.storm.scheduler.multitenant.MultitenantScheduler}.
+     * Round Robin Strategy uses this value to avoid spreading a topology too
+     * thinly over a large number of machines - avoiding the the extreme case where the topology would be spread over
+     * all workers and thus deny scheduling of other topologies. Round Robin scheduling will occupy all the workers on
+     * this limited number of machines, forcing other topologies to be scheduled on other machines; thus isolating the
+     * topology from other topologies.
+     * Set storm.scheduler to {@link org.apache.storm.scheduler.multitenant.MultitenantScheduler}
+     * Alternatively set storm.scheduler to {@link org.apache.storm.scheduler.resource.ResourceAwareScheduler}
+     * using {@link #TOPOLOGY_SCHEDULER_STRATEGY} set to
+     * {@link org.apache.storm.scheduler.resource.strategies.scheduling.RoundRobinResourceAwareStrategy}     */
     @IsInteger
     @IsPositiveNumber
     public static final String TOPOLOGY_ISOLATED_MACHINES = "topology.isolate.machines";
diff --git a/storm-core/test/jvm/org/apache/storm/messaging/NettyIntegrationTest.java b/storm-core/test/jvm/org/apache/storm/messaging/NettyIntegrationTest.java
index 867cc4e60..d581a81cc 100644
--- a/storm-core/test/jvm/org/apache/storm/messaging/NettyIntegrationTest.java
+++ b/storm-core/test/jvm/org/apache/storm/messaging/NettyIntegrationTest.java
@@ -91,8 +91,12 @@ public class NettyIntegrationTest {
             completeTopologyParams.setMockedSources(mockedSources);
 
             Map<String, List<FixedTuple>> results = Testing.completeTopology(cluster, topology, completeTopologyParams);
-
-            assertEquals(6 * 4, Testing.readTuples(results, "2").size());
+            List<List<Object>> tuplesRead = Testing.readTuples(results, "2");
+            String errMsg = "Tuples Read:\n\t"
+                    + String.join("\n\t", tuplesRead.stream().map(Object::toString).collect(Collectors.toList()))
+                    + "\nTuples Expected:\n\t"
+                    + String.join("\n\t", testTuples.stream().map(FixedTuple::toString).collect(Collectors.toList()));
+            assertEquals(6 * 4, tuplesRead.size(), errMsg);
         }
     }
 }
diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/BaseResourceAwareStrategy.java b/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/BaseResourceAwareStrategy.java
index 85937b985..94550a1fe 100644
--- a/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/BaseResourceAwareStrategy.java
+++ b/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/BaseResourceAwareStrategy.java
@@ -164,6 +164,7 @@ public abstract class BaseResourceAwareStrategy implements IStrategy {
 
         //order executors to be scheduled
         List<ExecutorDetails> orderedExecutors = execSorter.sortExecutors(unassignedExecutors);
+        isolateAckersToEnd(orderedExecutors);
         Iterable<String> sortedNodes = null;
         if (!this.sortNodesForEachExecutor) {
             nodeSorter.prepare(null);
@@ -403,20 +404,26 @@ public abstract class BaseResourceAwareStrategy implements IStrategy {
         return ret;
     }
 
+    /**
+     * Modify the collection, and place unassigned ackers to the end of the list.
+     *
+     * @param orderedExecutors List of executors that are presumed to be sorted.
+     */
+    private void isolateAckersToEnd(List<ExecutorDetails> orderedExecutors) {
+        orderedExecutors.removeAll(searcherState.getUnassignedAckers());
+        orderedExecutors.addAll(searcherState.getUnassignedAckers());
+        LOG.debug("For topology: {}, we have sorted execs: {} and unassigned ackers: {}",
+                topoName, orderedExecutors, searcherState.getUnassignedAckers());
+    }
+
     /**
      * Try to schedule till successful or till limits (backtrack count or time) have been exceeded.
      *
-     * @param orderedExecutors Executors sorted in the preferred order cannot be null.
+     * @param orderedExecutors Executors sorted in the preferred order cannot be null - note that ackers are isolated at the end.
      * @param sortedNodesIter Node iterable which may be null.
      * @return SchedulingResult with success attribute set to true or false indicting whether ALL executors were assigned.
      */
     protected SchedulingResult scheduleExecutorsOnNodes(List<ExecutorDetails> orderedExecutors, Iterable<String> sortedNodesIter) {
-        // isolate ackers and put it to the end of orderedExecutors
-        // the order of unassigned ackers in orderedExecutors and searcherState.getUnassignedAckers() are same
-        orderedExecutors.removeAll(searcherState.getUnassignedAckers());
-        orderedExecutors.addAll(searcherState.getUnassignedAckers());
-        LOG.debug("For topology: {}, we have sorted execs: {} and unassigned ackers: {}",
-                    topoName, orderedExecutors, searcherState.getUnassignedAckers());
 
         long         startTimeMilli     = Time.currentTimeMillis();
         searcherState.setSortedExecs(orderedExecutors);
@@ -487,7 +494,7 @@ public abstract class BaseResourceAwareStrategy implements IStrategy {
 
                     if (!isExecAssignmentToWorkerValid(exec, workerSlot)) {
                         // exec can't fit in this workerSlot, try next workerSlot
-                        LOG.debug("Failed to assign exec={}, comp={}, topo={} to worker={} on node=({}, availCpu={}, availMem={}).",
+                        LOG.trace("Failed to assign exec={}, comp={}, topo={} to worker={} on node=({}, availCpu={}, availMem={}).",
                             exec, comp, topoName, workerSlot,
                             node.getId(), node.getAvailableCpuResources(), node.getAvailableMemoryResources());
                         continue;
@@ -504,7 +511,7 @@ public abstract class BaseResourceAwareStrategy implements IStrategy {
                     if (searcherState.areAllExecsScheduled()) {
                         //Everything is scheduled correctly, so no need to search any more.
                         LOG.info("scheduleExecutorsOnNodes: Done at loopCnt={} in {}ms, state.elapsedtime={}, backtrackCnt={}, topo={}",
-                                loopCnt, System.currentTimeMillis() - startTimeMilli,
+                                loopCnt, Time.currentTimeMillis() - startTimeMilli,
                                 Time.currentTimeMillis() - searcherState.startTimeMillis,
                                 searcherState.getNumBacktrack(),
                                 topoName);
@@ -533,7 +540,7 @@ public abstract class BaseResourceAwareStrategy implements IStrategy {
         }
         boolean success = searcherState.areAllExecsScheduled();
         LOG.info("scheduleExecutorsOnNodes: Scheduled={} in {} milliseconds, state.elapsedtime={}, backtrackCnt={}, topo={}",
-                success, System.currentTimeMillis() - startTimeMilli, Time.currentTimeMillis() - searcherState.startTimeMillis,
+                success, Time.currentTimeMillis() - startTimeMilli, Time.currentTimeMillis() - searcherState.startTimeMillis,
                 searcherState.getNumBacktrack(),
                 topoName);
         return searcherState.createSchedulingResult(success, this.getClass().getSimpleName());
@@ -556,7 +563,7 @@ public abstract class BaseResourceAwareStrategy implements IStrategy {
      * @param workerSlot        WorkerSlot on which to schedule.
      * @return                  Number of ackers assigned.
      */
-    private int assignBoundAckersForNewWorkerSlot(ExecutorDetails exec, RasNode node, WorkerSlot workerSlot) {
+    protected int assignBoundAckersForNewWorkerSlot(ExecutorDetails exec, RasNode node, WorkerSlot workerSlot) {
         int numOfAckersToBind = searcherState.getNumOfAckersToBind(exec, workerSlot);
         if (numOfAckersToBind > 0) {
             for (int i = 0; i < numOfAckersToBind; i++) {
diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/ConstraintSolverConfig.java b/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/ConstraintSolverConfig.java
index 41eed696d..84dca6bc5 100644
--- a/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/ConstraintSolverConfig.java
+++ b/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/ConstraintSolverConfig.java
@@ -21,7 +21,6 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.storm.Config;
-import org.apache.storm.scheduler.ExecutorDetails;
 import org.apache.storm.scheduler.TopologyDetails;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/RoundRobinResourceAwareStrategy.java b/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/RoundRobinResourceAwareStrategy.java
new file mode 100644
index 000000000..c0bbcc3fc
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/RoundRobinResourceAwareStrategy.java
@@ -0,0 +1,187 @@
+/*
+ * 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.storm.scheduler.resource.strategies.scheduling;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.storm.Config;
+import org.apache.storm.scheduler.ExecutorDetails;
+import org.apache.storm.scheduler.WorkerSlot;
+import org.apache.storm.scheduler.resource.RasNode;
+import org.apache.storm.scheduler.resource.SchedulingResult;
+import org.apache.storm.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RoundRobinResourceAwareStrategy extends BaseResourceAwareStrategy {
+    private static final Logger LOG = LoggerFactory.getLogger(RoundRobinResourceAwareStrategy.class);
+
+    public RoundRobinResourceAwareStrategy() {
+        super(false, NodeSortType.COMMON);
+    }
+
+    /**
+     * Maximum number of isolated nodes being requested based on the topology configuration
+     * {@link Config#TOPOLOGY_ISOLATED_MACHINES}.
+     */
+    private int getMaxNumberOfNodesRequested() {
+        Map<String, Object> conf = topologyDetails.getConf();
+        if (conf.get(Config.TOPOLOGY_ISOLATED_MACHINES) == null) {
+            return Integer.MAX_VALUE;
+        } else {
+            return ((Number) topologyDetails.getConf().get(Config.TOPOLOGY_ISOLATED_MACHINES)).intValue();
+        }
+    }
+
+    /**
+     * If the number of machines is limited, then truncate the node list to this maximum number of nodes
+     * that have no other topologies running on it. If the current topology is running on it, then it
+     * is subject to selection in the list. If other topologies are running on it, then it is not selected.
+     *
+     * @param sortedNodesIterable Iterable of nodes
+     * @return an ArrayList of nodes
+     */
+    private ArrayList<String> getTruncatedNodeList(Iterable<String> sortedNodesIterable) {
+        final int maxNodes = getMaxNumberOfNodesRequested();
+        final ArrayList<String> ret = new ArrayList<>();
+        sortedNodesIterable.forEach(node -> {
+            if (ret.size() < maxNodes) {
+                RasNode rasNode = nodes.getNodeById(node);
+                Collection<String> runningTopos = rasNode.getRunningTopologies();
+                if (runningTopos.isEmpty() || runningTopos.size() == 1 && runningTopos.contains(topologyDetails.getId())) {
+                    ret.add(node);
+                }
+            }
+        });
+        return ret;
+    }
+
+    /**
+     * For each component try to schedule executors in sequence on the nodes.
+     *
+     * @param orderedExecutors Executors sorted in the preferred order cannot be null
+     * @param sortedNodesIterable Node iterable which cannot be null, relies on behavior when {@link #sortNodesForEachExecutor} is false
+     * @return SchedulingResult with success attribute set to true or false indicting whether ALL executors were assigned. @{#}
+     */
+    @Override
+    protected SchedulingResult scheduleExecutorsOnNodes(List<ExecutorDetails> orderedExecutors, Iterable<String> sortedNodesIterable) {
+        long startTimeMilli = Time.currentTimeMillis();
+        int  maxExecCnt     = searcherState.getExecSize();
+        int  nodeSortCnt    = 1;
+        Iterator<String> sortedNodesIter = null;
+        ArrayList<String> sortedNodes = getTruncatedNodeList(sortedNodesIterable);
+
+        LOG.debug("scheduleExecutorsOnNodes: will assign {} executors for topo {}", maxExecCnt, topoName);
+
+        searcherState.setSortedExecs(orderedExecutors);
+
+        OUTERMOST_LOOP:
+        for (int loopCnt = 0 ; true ; loopCnt++) {
+            LOG.debug("scheduleExecutorsOnNodes: loopCnt={}, execIndex={}, topo={}, nodeSortCnt={}",
+                    loopCnt, searcherState.getExecIndex(), topoName, nodeSortCnt);
+            if (searcherState.areSearchLimitsExceeded()) {
+                LOG.warn("Limits exceeded, loopCnt={}, topo={}, nodeSortCnt={}", loopCnt, topoName, nodeSortCnt);
+                return searcherState.createSchedulingResult(false, this.getClass().getSimpleName());
+            }
+
+            if (Thread.currentThread().isInterrupted()) {
+                return searcherState.createSchedulingResult(false, this.getClass().getSimpleName());
+            }
+
+            int execIndex = searcherState.getExecIndex();
+            ExecutorDetails exec = searcherState.currentExec();
+
+            // If current exec is found in searcherState assigned Ackers,
+            // it means it has been assigned as a bound acker already.
+            // So we skip to the next.
+            if (searcherState.getBoundAckers().contains(exec)) {
+                if (searcherState.areAllExecsScheduled()) {
+                    //Everything is scheduled correctly, so no need to search any more.
+                    LOG.info("scheduleExecutorsOnNodes: Done at loopCnt={} in {}ms, state.elapsedtime={}, topo={}, nodeSortCnt={}",
+                        loopCnt, Time.currentTimeMillis() - startTimeMilli,
+                        Time.currentTimeMillis() - searcherState.getStartTimeMillis(),
+                        topoName, nodeSortCnt);
+                    return searcherState.createSchedulingResult(true, this.getClass().getSimpleName());
+                }
+                searcherState = searcherState.nextExecutor();
+                continue OUTERMOST_LOOP;
+            }
+
+            String comp = execToComp.get(exec);
+            // start at the beginning of node list when component changes or when at end of nodes
+            if (sortedNodesIter == null || searcherState.isExecCompDifferentFromPrior() || !sortedNodesIter.hasNext()) {
+                sortedNodesIter = sortedNodes.iterator();
+                nodeSortCnt++;
+            }
+
+            while (sortedNodesIter.hasNext()) {
+                String nodeId = sortedNodesIter.next();
+                RasNode node = nodes.getNodeById(nodeId);
+                if (!node.couldEverFit(exec, topologyDetails)) {
+                    continue;
+                }
+                for (WorkerSlot workerSlot : node.getSlotsAvailableToScheduleOn()) {
+                    if (!isExecAssignmentToWorkerValid(exec, workerSlot)) {
+                        // exec can't fit in this workerSlot, try next workerSlot
+                        LOG.trace("Failed to assign exec={}, comp={}, topo={} to worker={} on node=({}, availCpu={}, availMem={}).",
+                            exec, comp, topoName, workerSlot,
+                            node.getId(), node.getAvailableCpuResources(), node.getAvailableMemoryResources());
+                        continue;
+                    }
+
+                    searcherState.incStatesSearched();
+                    searcherState.assignCurrentExecutor(execToComp, node, workerSlot);
+                    int numBoundAckerAssigned = assignBoundAckersForNewWorkerSlot(exec, node, workerSlot);
+                    if (numBoundAckerAssigned > 0) {
+                        // This exec with some of its bounded ackers have all been successfully assigned
+                        searcherState.getExecsWithBoundAckers().add(exec);
+                    }
+
+                    if (searcherState.areAllExecsScheduled()) {
+                        //Everything is scheduled correctly, so no need to search any more.
+                        LOG.info("scheduleExecutorsOnNodes: Done at loopCnt={} in {}ms, state.elapsedtime={}, topo={}, nodeSortCnt={}",
+                                loopCnt, Time.currentTimeMillis() - startTimeMilli,
+                                Time.currentTimeMillis() - searcherState.getStartTimeMillis(),
+                                topoName, nodeSortCnt);
+                        return searcherState.createSchedulingResult(true, this.getClass().getSimpleName());
+                    }
+                    searcherState = searcherState.nextExecutor();
+                    LOG.debug("scheduleExecutorsOnNodes: Assigned execId={}, comp={} to node={}/cpu={}/mem={}, "
+                            + "worker-port={} at loopCnt={}, topo={}, nodeSortCnt={}",
+                        execIndex, comp, nodeId, node.getAvailableCpuResources(), node.getAvailableMemoryResources(),
+                        workerSlot.getPort(), loopCnt, topoName, nodeSortCnt);
+                    continue OUTERMOST_LOOP;
+                }
+            }
+            // if here, then the executor was not assigned, scheduling failed
+            LOG.debug("scheduleExecutorsOnNodes: Failed to schedule execId={}, comp={} at loopCnt={}, topo={}, nodeSortCnt={}",
+                    execIndex, comp, loopCnt, topoName, nodeSortCnt);
+            break;
+        }
+        boolean success = searcherState.areAllExecsScheduled();
+        LOG.info("scheduleExecutorsOnNodes: Scheduled={} in {} milliseconds, state.elapsedtime={}, topo={}, nodeSortCnt={}",
+                success, Time.currentTimeMillis() - startTimeMilli, Time.currentTimeMillis() - searcherState.getStartTimeMillis(),
+                topoName, nodeSortCnt);
+        return searcherState.createSchedulingResult(success, this.getClass().getSimpleName());
+    }
+}
diff --git a/storm-server/src/test/java/org/apache/storm/TestRebalance.java b/storm-server/src/test/java/org/apache/storm/TestRebalance.java
index 53bd0719a..3800809d1 100644
--- a/storm-server/src/test/java/org/apache/storm/TestRebalance.java
+++ b/storm-server/src/test/java/org/apache/storm/TestRebalance.java
@@ -14,7 +14,6 @@ package org.apache.storm;
 
 import java.util.HashMap;
 import java.util.Map;
-import org.apache.storm.generated.AuthorizationException;
 import org.apache.storm.generated.ClusterSummary;
 import org.apache.storm.generated.RebalanceOptions;
 import org.apache.storm.generated.NotAliveException;
@@ -24,6 +23,10 @@ import org.apache.storm.scheduler.resource.ResourceAwareScheduler;
 import org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler;
 import org.apache.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy;
 import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategyOld;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategyOld;
+import org.apache.storm.scheduler.resource.strategies.scheduling.RoundRobinResourceAwareStrategy;
 import org.apache.storm.thrift.TException;
 import org.apache.storm.topology.BoltDeclarer;
 import org.apache.storm.topology.SpoutDeclarer;
@@ -39,6 +42,13 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestRebalance {
+    private static final Class[] strategyClasses = {
+            DefaultResourceAwareStrategy.class,
+            DefaultResourceAwareStrategyOld.class,
+            RoundRobinResourceAwareStrategy.class,
+            GenericResourceAwareStrategy.class,
+            GenericResourceAwareStrategyOld.class,
+    };
 
     static final int SLEEP_TIME_BETWEEN_RETRY = 1000;
 
@@ -54,93 +64,90 @@ public class TestRebalance {
         return null;
     }
 
-    protected Class getDefaultResourceAwareStrategyClass() {
-        return DefaultResourceAwareStrategy.class;
-    }
-
     @Test
     public void testRebalanceTopologyResourcesAndConfigs()
         throws Exception {
+        for (Class strategyClass : strategyClasses) {
+            LOG.info("Starting local cluster...using ", strategyClass.getName());
 
-        LOG.info("Starting local cluster...");
-
-        Config conf = new Config();
-        conf.put(DaemonConfig.STORM_SCHEDULER, ResourceAwareScheduler.class.getName());
-        conf.put(DaemonConfig.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, DefaultSchedulingPriorityStrategy.class.getName());
-        conf.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, getDefaultResourceAwareStrategyClass().getName());
-        conf.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 10.0);
-        conf.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 10.0);
-        conf.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 100.0);
-        conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE);
-        Map<String, Double> resourcesMap = new HashMap();
-        resourcesMap.put("gpu.count", 5.0);
-        conf.put(Config.TOPOLOGY_COMPONENT_RESOURCES_MAP, resourcesMap);
+            Config conf = new Config();
+            conf.put(DaemonConfig.STORM_SCHEDULER, ResourceAwareScheduler.class.getName());
+            conf.put(DaemonConfig.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, DefaultSchedulingPriorityStrategy.class.getName());
+            conf.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, strategyClass.getName());
+            conf.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 10.0);
+            conf.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 10.0);
+            conf.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 100.0);
+            conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE);
+            Map<String, Double> resourcesMap = new HashMap();
+            resourcesMap.put("gpu.count", 5.0);
+            conf.put(Config.TOPOLOGY_COMPONENT_RESOURCES_MAP, resourcesMap);
 
-        try (ILocalCluster cluster = new LocalCluster.Builder().withDaemonConf(conf).build()) {
+            try (ILocalCluster cluster = new LocalCluster.Builder().withDaemonConf(conf).build()) {
 
-            TopologyBuilder builder = new TopologyBuilder();
-            SpoutDeclarer s1 = builder.setSpout("spout-1", new TestUtilsForResourceAwareScheduler.TestSpout(),
-                                                2);
-            BoltDeclarer b1 = builder.setBolt("bolt-1", new TestUtilsForResourceAwareScheduler.TestBolt(),
-                                              2).shuffleGrouping("spout-1");
-            BoltDeclarer b2 = builder.setBolt("bolt-2", new TestUtilsForResourceAwareScheduler.TestBolt(),
-                                              2).shuffleGrouping("bolt-1");
+                TopologyBuilder builder = new TopologyBuilder();
+                SpoutDeclarer s1 = builder.setSpout("spout-1", new TestUtilsForResourceAwareScheduler.TestSpout(),
+                        2);
+                BoltDeclarer b1 = builder.setBolt("bolt-1", new TestUtilsForResourceAwareScheduler.TestBolt(),
+                        2).shuffleGrouping("spout-1");
+                BoltDeclarer b2 = builder.setBolt("bolt-2", new TestUtilsForResourceAwareScheduler.TestBolt(),
+                        2).shuffleGrouping("bolt-1");
 
-            StormTopology stormTopology = builder.createTopology();
+                StormTopology stormTopology = builder.createTopology();
 
-            LOG.info("submitting topologies....");
-            String topoName = "topo1";
-            cluster.submitTopology(topoName, new HashMap<>(), stormTopology);
+                LOG.info("submitting topologies....");
+                String topoName = "topo1";
+                cluster.submitTopology(topoName, new HashMap<>(), stormTopology);
 
-            waitTopologyScheduled(topoName, cluster, 20);
+                waitTopologyScheduled(topoName, cluster, 20);
 
-            RebalanceOptions opts = new RebalanceOptions();
+                RebalanceOptions opts = new RebalanceOptions();
 
-            Map<String, Map<String, Double>> resources = new HashMap<String, Map<String, Double>>();
-            resources.put("spout-1", new HashMap<String, Double>());
-            resources.get("spout-1").put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 120.0);
-            resources.get("spout-1").put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 25.0);
-            resources.get("spout-1").put("gpu.count", 5.0);
+                Map<String, Map<String, Double>> resources = new HashMap<String, Map<String, Double>>();
+                resources.put("spout-1", new HashMap<String, Double>());
+                resources.get("spout-1").put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 120.0);
+                resources.get("spout-1").put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 25.0);
+                resources.get("spout-1").put("gpu.count", 5.0);
 
-            opts.set_topology_resources_overrides(resources);
-            opts.set_wait_secs(0);
+                opts.set_topology_resources_overrides(resources);
+                opts.set_wait_secs(0);
 
-            JSONObject jsonObject = new JSONObject();
-            jsonObject.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 768.0);
+                JSONObject jsonObject = new JSONObject();
+                jsonObject.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 768.0);
 
-            opts.set_topology_conf_overrides(jsonObject.toJSONString());
+                opts.set_topology_conf_overrides(jsonObject.toJSONString());
 
-            LOG.info("rebalancing....");
-            cluster.rebalance("topo1", opts);
+                LOG.info("rebalancing....");
+                cluster.rebalance("topo1", opts);
 
-            waitTopologyScheduled(topoName, cluster, 10);
+                waitTopologyScheduled(topoName, cluster, 10);
 
-            boolean topologyUpdated = false;
-            JSONParser parser = new JSONParser();
+                boolean topologyUpdated = false;
+                JSONParser parser = new JSONParser();
 
-            for (int i = 0; i < 5; i++) {
-                Utils.sleep(SLEEP_TIME_BETWEEN_RETRY);
+                for (int i = 0; i < 5; i++) {
+                    Utils.sleep(SLEEP_TIME_BETWEEN_RETRY);
 
-                String confRaw = cluster.getTopologyConf(topoNameToId(topoName, cluster));
+                    String confRaw = cluster.getTopologyConf(topoNameToId(topoName, cluster));
 
 
-                JSONObject readConf = (JSONObject) parser.parse(confRaw);
-                if (768.0 == (double) readConf.get(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB)) {
-                    topologyUpdated = true;
-                    break;
+                    JSONObject readConf = (JSONObject) parser.parse(confRaw);
+                    if (768.0 == (double) readConf.get(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB)) {
+                        topologyUpdated = true;
+                        break;
+                    }
                 }
-            }
 
-            StormTopology readStormTopology = cluster.getTopology(topoNameToId(topoName, cluster));
-            String componentConfRaw = readStormTopology.get_spouts().get("spout-1").get_common().get_json_conf();
+                StormTopology readStormTopology = cluster.getTopology(topoNameToId(topoName, cluster));
+                String componentConfRaw = readStormTopology.get_spouts().get("spout-1").get_common().get_json_conf();
 
-            JSONObject readTopologyConf = (JSONObject) parser.parse(componentConfRaw);
+                JSONObject readTopologyConf = (JSONObject) parser.parse(componentConfRaw);
 
-            Map<String, Double> componentResources = (Map<String, Double>) readTopologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_MAP);
-            assertTrue(topologyUpdated, "Topology has been updated");
-            assertEquals(25.0, componentResources.get(Constants.COMMON_CPU_RESOURCE_NAME), 0.001, "Updated CPU correct");
-            assertEquals(120.0, componentResources.get(Constants.COMMON_ONHEAP_MEMORY_RESOURCE_NAME), 0.001, "Updated Memory correct");
-            assertEquals(5.0, componentResources.get("gpu.count"), 0.001, "Updated Generic resource correct");
+                Map<String, Double> componentResources = (Map<String, Double>) readTopologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_MAP);
+                assertTrue(topologyUpdated, "Topology has been updated");
+                assertEquals(25.0, componentResources.get(Constants.COMMON_CPU_RESOURCE_NAME), 0.001, "Updated CPU correct");
+                assertEquals(120.0, componentResources.get(Constants.COMMON_ONHEAP_MEMORY_RESOURCE_NAME), 0.001, "Updated Memory correct");
+                assertEquals(5.0, componentResources.get("gpu.count"), 0.001, "Updated Generic resource correct");
+            }
         }
     }
 
diff --git a/storm-server/src/test/java/org/apache/storm/daemon/nimbus/NimbusTest.java b/storm-server/src/test/java/org/apache/storm/daemon/nimbus/NimbusTest.java
index 187ad0d52..002827ed4 100644
--- a/storm-server/src/test/java/org/apache/storm/daemon/nimbus/NimbusTest.java
+++ b/storm-server/src/test/java/org/apache/storm/daemon/nimbus/NimbusTest.java
@@ -29,6 +29,8 @@ import org.apache.storm.generated.InvalidTopologyException;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy;
 import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategyOld;
+import org.apache.storm.scheduler.resource.strategies.scheduling.RoundRobinResourceAwareStrategy;
 import org.apache.storm.testing.TestWordSpout;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.utils.ServerUtils;
@@ -41,10 +43,6 @@ import static org.junit.jupiter.api.Assertions.fail;
 import static org.junit.jupiter.api.Assertions.assertNull;
 
 public class NimbusTest {
-    protected Class getDefaultResourceAwareStrategyClass() {
-        return DefaultResourceAwareStrategy.class;
-    }
-
     @Test
     public void testMemoryLoadLargerThanMaxHeapSize() {
         // Topology will not be able to be successfully scheduled: Config TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB=128.0 < 129.0,
@@ -56,18 +54,25 @@ public class NimbusTest {
         config1.put(Config.STORM_NETWORK_TOPOGRAPHY_PLUGIN, "org.apache.storm.networktopography.DefaultRackDNSToSwitchMapping");
         config1.put(DaemonConfig.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, DefaultSchedulingPriorityStrategy.class.getName());
 
-        config1.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, getDefaultResourceAwareStrategyClass().getName());
         config1.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 10.0);
         config1.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 0.0);
         config1.put(Config.TOPOLOGY_PRIORITY, 0);
         config1.put(Config.TOPOLOGY_SUBMITTER_USER, "zhuo");
         config1.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 128.0);
         config1.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 129.0);
-        try {
-            ServerUtils.validateTopologyWorkerMaxHeapSizeConfigs(config1, stormTopology1, 768.0);
-            fail("Expected exception not thrown");
-        } catch (InvalidTopologyException e) {
-            //Expected...
+        Class[] strategyClasses = {
+                DefaultResourceAwareStrategy.class,
+                RoundRobinResourceAwareStrategy.class,
+                GenericResourceAwareStrategyOld.class};
+        for (Class strategyClass: strategyClasses) {
+            String strategyClassName = strategyClass.getName();
+            config1.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, strategyClassName);
+            try {
+                ServerUtils.validateTopologyWorkerMaxHeapSizeConfigs(config1, stormTopology1, 768.0);
+                fail("Expected exception not thrown when using Strategy " + strategyClassName);
+            } catch (InvalidTopologyException e) {
+                //Expected...
+            }
         }
     }
 
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/blacklist/TestBlacklistScheduler.java b/storm-server/src/test/java/org/apache/storm/scheduler/blacklist/TestBlacklistScheduler.java
index 8353c7d68..bed28ce21 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/blacklist/TestBlacklistScheduler.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/blacklist/TestBlacklistScheduler.java
@@ -32,6 +32,10 @@ import org.apache.storm.scheduler.Topologies;
 import org.apache.storm.scheduler.TopologyDetails;
 import org.apache.storm.scheduler.resource.ResourceAwareScheduler;
 import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategyOld;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategyOld;
+import org.apache.storm.scheduler.resource.strategies.scheduling.RoundRobinResourceAwareStrategy;
 import org.apache.storm.utils.Utils;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.Test;
@@ -51,6 +55,7 @@ import org.apache.storm.metric.StormMetricsRegistry;
 import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestBlacklistScheduler {
@@ -60,10 +65,6 @@ public class TestBlacklistScheduler {
     private int currentTime = 1468216504;
     private IScheduler scheduler = null;
 
-    protected Class getDefaultResourceAwareStrategyClass() {
-        return DefaultResourceAwareStrategy.class;
-    }
-
     @AfterEach
     public void cleanup() {
         if (scheduler != null) {
@@ -244,39 +245,56 @@ public class TestBlacklistScheduler {
         config.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 0.0);
         config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 0);
         config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 0);
-        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, getDefaultResourceAwareStrategyClass().getName());
         config.put(Config.TOPOLOGY_RAS_ONE_EXECUTOR_PER_WORKER, true);
 
-        Map<String, TopologyDetails> topoMap = new HashMap<>();
-
-        TopologyDetails topo1 = TestUtilsForBlacklistScheduler.getTopology("topo-1", config, 1, 1, 1, 1, currentTime - 2, true);
-        TopologyDetails topo2 = TestUtilsForBlacklistScheduler.getTopology("topo-2", config, 1, 1, 1, 1, currentTime - 8, true);
-        Topologies topologies = new Topologies(topoMap);
-
-        StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
-        ResourceMetrics resourceMetrics = new ResourceMetrics(metricsRegistry);
-        Cluster cluster = new Cluster(iNimbus, resourceMetrics, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
-        scheduler = new BlacklistScheduler(new ResourceAwareScheduler());
-
-        scheduler.prepare(config, metricsRegistry);
-        scheduler.schedule(topologies, cluster);
-        cluster = new Cluster(iNimbus, resourceMetrics, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
-        scheduler.schedule(topologies, cluster);
-        cluster = new Cluster(iNimbus, resourceMetrics, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
-        scheduler.schedule(topologies, cluster);
-        cluster = new Cluster(iNimbus, resourceMetrics, supMap, TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
-        scheduler.schedule(topologies, cluster);
-        assertEquals(Collections.singleton("host-0"), cluster.getBlacklistedHosts(), "blacklist");
-
-        topoMap.put(topo1.getId(), topo1);
-        topoMap.put(topo2.getId(), topo2);
-        topologies = new Topologies(topoMap);
-        cluster = new Cluster(iNimbus, resourceMetrics, supMap, TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
-        scheduler.schedule(topologies, cluster);
-        assertEquals(Collections.emptySet(), cluster.getBlacklistedHosts(), "blacklist");
-        assertEquals(Collections.singletonList("sup-0"), cluster.getGreyListedSupervisors(), "greylist");
-        LOG.debug("Now only these slots remain available: {}", cluster.getAvailableSlots());
-        assertTrue(cluster.getAvailableSlots(supMap.get("sup-0")).containsAll(cluster.getAvailableSlots()));
+        Class[] strategyClasses = {
+                DefaultResourceAwareStrategy.class,
+                DefaultResourceAwareStrategyOld.class,
+                RoundRobinResourceAwareStrategy.class,
+                GenericResourceAwareStrategy.class,
+                GenericResourceAwareStrategyOld.class,
+        };
+        for (Class strategyClass: strategyClasses) {
+            String strategyClassName = strategyClass.getName();
+            config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, strategyClassName);
+            {
+                Map<String, TopologyDetails> topoMap = new HashMap<>();
+
+                TopologyDetails topo1 = TestUtilsForBlacklistScheduler.getTopology("topo-1", config, 1, 1, 1, 1, currentTime - 2, true);
+                TopologyDetails topo2 = TestUtilsForBlacklistScheduler.getTopology("topo-2", config, 1, 1, 1, 1, currentTime - 8, true);
+                Topologies topologies = new Topologies(topoMap);
+
+                StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
+                ResourceMetrics resourceMetrics = new ResourceMetrics(metricsRegistry);
+                Cluster cluster = new Cluster(iNimbus, resourceMetrics, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+                scheduler = new BlacklistScheduler(new ResourceAwareScheduler());
+
+                scheduler.prepare(config, metricsRegistry);
+                scheduler.schedule(topologies, cluster);
+                cluster = new Cluster(iNimbus, resourceMetrics, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+                scheduler.schedule(topologies, cluster);
+                cluster = new Cluster(iNimbus, resourceMetrics, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+                scheduler.schedule(topologies, cluster);
+                cluster = new Cluster(iNimbus, resourceMetrics, supMap, TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+                scheduler.schedule(topologies, cluster);
+                assertEquals(Collections.singleton("host-0"), cluster.getBlacklistedHosts(), "blacklist");
+
+                topoMap.put(topo1.getId(), topo1);
+                topoMap.put(topo2.getId(), topo2);
+                topologies = new Topologies(topoMap);
+                cluster = new Cluster(iNimbus, resourceMetrics, supMap, TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+                scheduler.schedule(topologies, cluster);
+                assertEquals(Collections.emptySet(), cluster.getBlacklistedHosts(), "blacklist using " + strategyClassName);
+                assertEquals(Collections.singletonList("sup-0"), cluster.getGreyListedSupervisors(), "greylist using" +  strategyClassName);
+                LOG.debug("{}: Now only these slots remain available: {}", strategyClassName, cluster.getAvailableSlots());
+                if (strategyClass == RoundRobinResourceAwareStrategy.class) {
+                    // available slots will be across supervisors
+                    assertFalse(cluster.getAvailableSlots(supMap.get("sup-0")).containsAll(cluster.getAvailableSlots()), "using " + strategyClassName);
+                } else {
+                    assertTrue(cluster.getAvailableSlots(supMap.get("sup-0")).containsAll(cluster.getAvailableSlots()), "using " + strategyClassName);
+                }
+            }
+        }
     }
 
     @Test
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestResourceAwareScheduler.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestResourceAwareScheduler.java
index 5355a4e8e..0a69f66e1 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestResourceAwareScheduler.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestResourceAwareScheduler.java
@@ -12,6 +12,7 @@
 
 package org.apache.storm.scheduler.resource;
 
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -30,6 +31,7 @@ import org.apache.storm.Config;
 import org.apache.storm.DaemonConfig;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.generated.WorkerResources;
+import org.apache.storm.metric.StormMetricsRegistry;
 import org.apache.storm.scheduler.Cluster;
 import org.apache.storm.scheduler.ExecutorDetails;
 import org.apache.storm.scheduler.INimbus;
@@ -46,6 +48,9 @@ import org.apache.storm.scheduler.resource.strategies.scheduling.ConstraintSolve
 import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy;
 import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategyOld;
 import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategyOld;
+import org.apache.storm.scheduler.resource.strategies.scheduling.RoundRobinResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
 import org.apache.storm.testing.PerformanceTest;
 import org.apache.storm.testing.TestWordCounter;
 import org.apache.storm.testing.TestWordSpout;
@@ -57,6 +62,8 @@ import org.apache.storm.utils.Time;
 import org.apache.storm.utils.Utils;
 import org.apache.storm.validation.ConfigValidation;
 import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -65,44 +72,29 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 
-import java.time.Duration;
-import org.apache.storm.metric.StormMetricsRegistry;
-import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
 
 public class TestResourceAwareScheduler {
 
     private static final Logger LOG = LoggerFactory.getLogger(TestResourceAwareScheduler.class);
+    private static final Class[] strategyClasses = {
+            DefaultResourceAwareStrategy.class,
+            RoundRobinResourceAwareStrategy.class,
+            GenericResourceAwareStrategy.class,
+    };
     private final Config defaultTopologyConf;
     private int currentTime = 1450418597;
     private IScheduler scheduler = null;
 
     public TestResourceAwareScheduler() {
-        defaultTopologyConf = createClusterConfig(10, 128, 0, null);
+        defaultTopologyConf = createClusterConfig(DefaultResourceAwareStrategy.class, 10, 128, 0, null);
         defaultTopologyConf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 8192.0);
         defaultTopologyConf.put(Config.TOPOLOGY_PRIORITY, 0);
     }
 
-    protected Class getDefaultResourceAwareStrategyClass() {
-        return DefaultResourceAwareStrategy.class;
-    }
-
-    protected Class getGenericResourceAwareStrategyClass() {
-        return GenericResourceAwareStrategy.class;
-    }
-
-    private Config createGrasClusterConfig(double compPcore, double compOnHeap, double compOffHeap,
-                                           Map<String, Map<String, Number>> pools, Map<String, Double> genericResourceMap) {
-        Config config = TestUtilsForResourceAwareScheduler.createGrasClusterConfig(compPcore, compOnHeap, compOffHeap, pools, genericResourceMap);
-        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, getGenericResourceAwareStrategyClass().getName());
-        return config;
-    }
-
-    private Config createClusterConfig(double compPcore, double compOnHeap, double compOffHeap,
+    private Config createClusterConfig(Class strategyClass, double compPcore, double compOnHeap, double compOffHeap,
                                        Map<String, Map<String, Number>> pools) {
         Config config = TestUtilsForResourceAwareScheduler.createClusterConfig(compPcore, compOnHeap, compOffHeap, pools);
-        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, getDefaultResourceAwareStrategyClass().getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, strategyClass.getName());
         return config;
     }
 
@@ -712,14 +704,14 @@ public class TestResourceAwareScheduler {
 
     @Test
     public void testHeterogeneousClusterwithDefaultRas() {
-        testHeterogeneousCluster(defaultTopologyConf, getDefaultResourceAwareStrategyClass().getSimpleName());
+        testHeterogeneousCluster(defaultTopologyConf, DefaultResourceAwareStrategy.class.getSimpleName());
     }
 
     @Test
     public void testHeterogeneousClusterwithGras() {
         Config grasClusterConfig = (Config) defaultTopologyConf.clone();
-        grasClusterConfig.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, getGenericResourceAwareStrategyClass().getName());
-        testHeterogeneousCluster(grasClusterConfig, getGenericResourceAwareStrategyClass().getSimpleName());
+        grasClusterConfig.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, GenericResourceAwareStrategy.class.getName());
+        testHeterogeneousCluster(grasClusterConfig, GenericResourceAwareStrategy.class.getSimpleName());
     }
 
     @Test
@@ -791,79 +783,87 @@ public class TestResourceAwareScheduler {
         Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
             userRes("jerry", 200, 2000));
 
-        Config config = createClusterConfig(100, 500, 500, resourceUserPool);
+        for (Class strategyClass: strategyClasses) {
+            Config config = createClusterConfig(strategyClass, 100, 500, 500, resourceUserPool);
 
-        Topologies topologies = new Topologies(
-            genTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 10, "jerry"),
-            genTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"),
-            genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"),
-            genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
-            genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 20, "bobby"));
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+            Topologies topologies = new Topologies(
+                    genTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 10, "jerry"),
+                    genTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"),
+                    genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"),
+                    genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
+                    genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 20, "bobby"));
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
 
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-        assertTopologiesFullyScheduled(cluster, "topo-1", "topo-2", "topo-3", "topo-4");
-        assertTopologiesNotScheduled(cluster, "topo-5");
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+            assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1", "topo-2", "topo-3", "topo-4");
+            assertTopologiesNotScheduled(cluster, strategyClass, "topo-5");
+        }
     }
 
     @Test
     public void testMultipleUsers() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(20, 4, 1000, 1024 * 10);
-        Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
-            userRes("jerry", 1_000, 8_192),
-            userRes("bobby", 10_000, 32_768),
-            userRes("derek", 5_000, 16_384));
-        Config config = createClusterConfig(10, 128, 0, resourceUserPool);
-
-        Topologies topologies = new Topologies(
-            genTopology("topo-1", config, 5, 15, 1, 1, currentTime - 2, 20, "jerry"),
-            genTopology("topo-2", config, 5, 15, 1, 1, currentTime - 8, 29, "jerry"),
-            genTopology("topo-3", config, 5, 15, 1, 1, currentTime - 16, 29, "jerry"),
-            genTopology("topo-4", config, 5, 15, 1, 1, currentTime - 16, 20, "jerry"),
-            genTopology("topo-5", config, 5, 15, 1, 1, currentTime - 24, 29, "jerry"),
-            genTopology("topo-6", config, 5, 15, 1, 1, currentTime - 2, 20, "bobby"),
-            genTopology("topo-7", config, 5, 15, 1, 1, currentTime - 8, 29, "bobby"),
-            genTopology("topo-8", config, 5, 15, 1, 1, currentTime - 16, 29, "bobby"),
-            genTopology("topo-9", config, 5, 15, 1, 1, currentTime - 16, 20, "bobby"),
-            genTopology("topo-10", config, 5, 15, 1, 1, currentTime - 24, 29, "bobby"),
-            genTopology("topo-11", config, 5, 15, 1, 1, currentTime - 2, 20, "derek"),
-            genTopology("topo-12", config, 5, 15, 1, 1, currentTime - 8, 29, "derek"),
-            genTopology("topo-13", config, 5, 15, 1, 1, currentTime - 16, 29, "derek"),
-            genTopology("topo-14", config, 5, 15, 1, 1, currentTime - 16, 20, "derek"),
-            genTopology("topo-15", config, 5, 15, 1, 1, currentTime - 24, 29, "derek"));
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+        for (Class strategyClass: strategyClasses) {
+            if (strategyClass.getName().equals(RoundRobinResourceAwareStrategy.class.getName())) {
+                continue; // exclude RoundRbin from this test
+            }
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(20, 4, 1000, 1024 * 10);
+            Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
+                    userRes("jerry", 1_000, 8_192),
+                    userRes("bobby", 10_000, 32_768),
+                    userRes("derek", 5_000, 16_384));
+            Config config = createClusterConfig(strategyClass, 10, 128, 0, resourceUserPool);
+            TopologyDetails[] topos = {
+                    genTopology("topo-1", config, 5, 15, 1, 1, currentTime - 2, 20, "jerry"),
+                    genTopology("topo-2", config, 5, 15, 1, 1, currentTime - 8, 29, "jerry"),
+                    genTopology("topo-3", config, 5, 15, 1, 1, currentTime - 16, 29, "jerry"),
+                    genTopology("topo-4", config, 5, 15, 1, 1, currentTime - 16, 20, "jerry"),
+                    genTopology("topo-5", config, 5, 15, 1, 1, currentTime - 24, 29, "jerry"),
+                    genTopology("topo-6", config, 5, 15, 1, 1, currentTime - 2, 20, "bobby"),
+                    genTopology("topo-7", config, 5, 15, 1, 1, currentTime - 8, 29, "bobby"),
+                    genTopology("topo-8", config, 5, 15, 1, 1, currentTime - 16, 29, "bobby"),
+                    genTopology("topo-9", config, 5, 15, 1, 1, currentTime - 16, 20, "bobby"),
+                    genTopology("topo-10", config, 5, 15, 1, 1, currentTime - 24, 29, "bobby"),
+                    genTopology("topo-11", config, 5, 15, 1, 1, currentTime - 2, 20, "derek"),
+                    genTopology("topo-12", config, 5, 15, 1, 1, currentTime - 8, 29, "derek"),
+                    genTopology("topo-13", config, 5, 15, 1, 1, currentTime - 16, 29, "derek"),
+                    genTopology("topo-14", config, 5, 15, 1, 1, currentTime - 16, 20, "derek"),
+                    genTopology("topo-15", config, 5, 15, 1, 1, currentTime - 24, 29, "derek"),
+            };
+            Topologies topologies = new Topologies(topos);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
 
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-        for (TopologyDetails td : topologies) {
-            assertTopologiesFullyScheduled(cluster, td.getName());
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+            assertTopologiesFullyScheduled(cluster, strategyClass, topos.length);
         }
     }
 
     @Test
     public void testHandlingClusterSubscription() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(1, 4, 200, 1024 * 10);
-        Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
-            userRes("jerry", 1_000, 8_192),
-            userRes("bobby", 10_000, 32_768),
-            userRes("derek", 5_000, 16_384));
-        Config config = createClusterConfig(10, 128, 0, resourceUserPool);
-
-        Topologies topologies = new Topologies(
-            genTopology("topo-1", config, 5, 15, 1, 1, currentTime - 2, 20, "jerry"),
-            genTopology("topo-2", config, 5, 15, 1, 1, currentTime - 8, 29, "jerry"));
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
+        for (Class strategyClass: strategyClasses) {
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(1, 4, 200, 1024 * 10);
+            Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
+                    userRes("jerry", 1_000, 8_192),
+                    userRes("bobby", 10_000, 32_768),
+                    userRes("derek", 5_000, 16_384));
+            Config config = createClusterConfig(strategyClass, 10, 128, 0, resourceUserPool);
+
+            Topologies topologies = new Topologies(
+                    genTopology("topo-1", config, 5, 15, 1, 1, currentTime - 2, 20, "jerry"),
+                    genTopology("topo-2", config, 5, 15, 1, 1, currentTime - 8, 29, "jerry"));
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
 
-        assertTopologiesFullyScheduled(cluster, "topo-1");
-        assertTopologiesNotScheduled(cluster, "topo-2");
+            assertTopologiesFullyScheduled(cluster, strategyClass,"topo-1");
+            assertTopologiesNotScheduled(cluster, strategyClass, "topo-2");
+        }
     }
 
     /**
@@ -872,55 +872,57 @@ public class TestResourceAwareScheduler {
      */
     @Test
     public void testFaultTolerance() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(6, 4, 100, 1000);
-        Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
-            userRes("jerry", 50, 500),
-            userRes("bobby", 200, 2_000),
-            userRes("derek", 100, 1_000));
-        Config config = createClusterConfig(100, 500, 500, resourceUserPool);
-
-        Topologies topologies = new Topologies(
-            genTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 21, "jerry"),
-            genTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"),
-            genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
-            genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
-            genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"),
-            genTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 10, "derek"));
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-        scheduler = new ResourceAwareScheduler();
+        for (Class strategyClass: strategyClasses) {
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(6, 4, 100, 1000);
+            Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
+                    userRes("jerry", 50, 500),
+                    userRes("bobby", 200, 2_000),
+                    userRes("derek", 100, 1_000));
+            Config config = createClusterConfig(strategyClass, 100, 500, 500, resourceUserPool);
+
+            Topologies topologies = new Topologies(
+                    genTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 21, "jerry"),
+                    genTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"),
+                    genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
+                    genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
+                    genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"),
+                    genTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 10, "derek"));
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+            scheduler = new ResourceAwareScheduler();
 
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
 
-        assertTopologiesFullyScheduled(cluster, "topo-1", "topo-2", "topo-3", "topo-4", "topo-5", "topo-6");
-
-        //fail supervisor
-        SupervisorDetails supFailed = cluster.getSupervisors().values().iterator().next();
-        LOG.info("/***** failing supervisor: {} ****/", supFailed.getHost());
-        supMap.remove(supFailed.getId());
-        Map<String, SchedulerAssignmentImpl> newAssignments = new HashMap<>();
-        for (Map.Entry<String, SchedulerAssignment> topoToAssignment : cluster.getAssignments().entrySet()) {
-            String topoId = topoToAssignment.getKey();
-            SchedulerAssignment assignment = topoToAssignment.getValue();
-            Map<ExecutorDetails, WorkerSlot> executorToSlots = new HashMap<>();
-            for (Map.Entry<ExecutorDetails, WorkerSlot> execToWorker : assignment.getExecutorToSlot().entrySet()) {
-                ExecutorDetails exec = execToWorker.getKey();
-                WorkerSlot ws = execToWorker.getValue();
-                if (!ws.getNodeId().equals(supFailed.getId())) {
-                    executorToSlots.put(exec, ws);
+            assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1", "topo-2", "topo-3", "topo-4", "topo-5", "topo-6");
+
+            //fail supervisor
+            SupervisorDetails supFailed = cluster.getSupervisors().values().iterator().next();
+            LOG.info("/***** failing supervisor: {} ****/", supFailed.getHost());
+            supMap.remove(supFailed.getId());
+            Map<String, SchedulerAssignmentImpl> newAssignments = new HashMap<>();
+            for (Map.Entry<String, SchedulerAssignment> topoToAssignment : cluster.getAssignments().entrySet()) {
+                String topoId = topoToAssignment.getKey();
+                SchedulerAssignment assignment = topoToAssignment.getValue();
+                Map<ExecutorDetails, WorkerSlot> executorToSlots = new HashMap<>();
+                for (Map.Entry<ExecutorDetails, WorkerSlot> execToWorker : assignment.getExecutorToSlot().entrySet()) {
+                    ExecutorDetails exec = execToWorker.getKey();
+                    WorkerSlot ws = execToWorker.getValue();
+                    if (!ws.getNodeId().equals(supFailed.getId())) {
+                        executorToSlots.put(exec, ws);
+                    }
                 }
+                newAssignments.put(topoId, new SchedulerAssignmentImpl(topoId, executorToSlots, null, null));
             }
-            newAssignments.put(topoId, new SchedulerAssignmentImpl(topoId, executorToSlots, null, null));
-        }
-        Map<String, String> statusMap = cluster.getStatusMap();
-        LOG.warn("Rescheduling with removed Supervisor....");
-        cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, newAssignments, topologies, config);
-        cluster.setStatusMap(statusMap);
-        scheduler.schedule(topologies, cluster);
+            Map<String, String> statusMap = cluster.getStatusMap();
+            LOG.warn("Rescheduling with removed Supervisor....");
+            cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, newAssignments, topologies, config);
+            cluster.setStatusMap(statusMap);
+            scheduler.schedule(topologies, cluster);
 
-        assertTopologiesFullyScheduled(cluster, "topo-2", "topo-3", "topo-4", "topo-5", "topo-6");
-        assertTopologiesNotScheduled(cluster, "topo-1");
+            assertTopologiesFullyScheduled(cluster, strategyClass, "topo-2", "topo-3", "topo-4", "topo-5", "topo-6");
+            assertTopologiesNotScheduled(cluster, strategyClass, "topo-1");
+        }
     }
 
     /**
@@ -928,37 +930,39 @@ public class TestResourceAwareScheduler {
      */
     @Test
     public void testNodeFreeSlot() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100, 1000);
-        Config config = createClusterConfig(100, 500, 500, null);
-
-        Topologies topologies = new Topologies(
-            genTopology("topo-1", config, 1, 0, 2, 0, currentTime - 2, 29, "user"),
-            genTopology("topo-2", config, 1, 0, 2, 0, currentTime - 2, 10, "user"));
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-        scheduler = new ResourceAwareScheduler();
-
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
+        for (Class strategyClass: strategyClasses) {
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100, 1000);
+            Config config = createClusterConfig(strategyClass, 100, 500, 500, null);
+
+            Topologies topologies = new Topologies(
+                    genTopology("topo-1", config, 1, 0, 2, 0, currentTime - 2, 29, "user"),
+                    genTopology("topo-2", config, 1, 0, 2, 0, currentTime - 2, 10, "user"));
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+            scheduler = new ResourceAwareScheduler();
 
-        Map<String, RasNode> nodes = RasNodes.getAllNodesFrom(cluster);
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
 
-        for (SchedulerAssignment assignment : cluster.getAssignments().values()) {
-            for (Entry<WorkerSlot, WorkerResources> entry : new HashMap<>(assignment.getScheduledResources()).entrySet()) {
-                WorkerSlot ws = entry.getKey();
-                WorkerResources wr = entry.getValue();
-                double memoryBefore = nodes.get(ws.getNodeId()).getAvailableMemoryResources();
-                double cpuBefore = nodes.get(ws.getNodeId()).getAvailableCpuResources();
-                double memoryUsedByWorker = wr.get_mem_on_heap() + wr.get_mem_off_heap();
-                assertEquals(1000.0, memoryUsedByWorker, 0.001, "Check if memory used by worker is calculated correctly");
-                double cpuUsedByWorker = wr.get_cpu();
-                assertEquals(100.0, cpuUsedByWorker, 0.001, "Check if CPU used by worker is calculated correctly");
-                nodes.get(ws.getNodeId()).free(ws);
-                double memoryAfter = nodes.get(ws.getNodeId()).getAvailableMemoryResources();
-                double cpuAfter = nodes.get(ws.getNodeId()).getAvailableCpuResources();
-                assertEquals(memoryBefore + memoryUsedByWorker, memoryAfter, 0.001, "Check if free correctly frees amount of memory");
-                assertEquals(cpuBefore + cpuUsedByWorker, cpuAfter, 0.001, "Check if free correctly frees amount of memory");
-                assertFalse(assignment.getSlotToExecutors().containsKey(ws), "Check if worker was removed from assignments");
+            Map<String, RasNode> nodes = RasNodes.getAllNodesFrom(cluster);
+
+            for (SchedulerAssignment assignment : cluster.getAssignments().values()) {
+                for (Entry<WorkerSlot, WorkerResources> entry : new HashMap<>(assignment.getScheduledResources()).entrySet()) {
+                    WorkerSlot ws = entry.getKey();
+                    WorkerResources wr = entry.getValue();
+                    double memoryBefore = nodes.get(ws.getNodeId()).getAvailableMemoryResources();
+                    double cpuBefore = nodes.get(ws.getNodeId()).getAvailableCpuResources();
+                    double memoryUsedByWorker = wr.get_mem_on_heap() + wr.get_mem_off_heap();
+                    assertEquals(1000.0, memoryUsedByWorker, 0.001, "Check if memory used by worker is calculated correctly");
+                    double cpuUsedByWorker = wr.get_cpu();
+                    assertEquals(100.0, cpuUsedByWorker, 0.001, "Check if CPU used by worker is calculated correctly");
+                    nodes.get(ws.getNodeId()).free(ws);
+                    double memoryAfter = nodes.get(ws.getNodeId()).getAvailableMemoryResources();
+                    double cpuAfter = nodes.get(ws.getNodeId()).getAvailableCpuResources();
+                    assertEquals(memoryBefore + memoryUsedByWorker, memoryAfter, 0.001, "Check if free correctly frees amount of memory");
+                    assertEquals(cpuBefore + cpuUsedByWorker, cpuAfter, 0.001, "Check if free correctly frees amount of memory");
+                    assertFalse(assignment.getSlotToExecutors().containsKey(ws), "Check if worker was removed from assignments");
+                }
             }
         }
     }
@@ -968,26 +972,28 @@ public class TestResourceAwareScheduler {
      */
     @Test
     public void testSchedulingAfterFailedScheduling() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(8, 4, 100, 1000);
-        Config config = createClusterConfig(100, 500, 500, null);
+        for (Class strategyClass: strategyClasses) {
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(8, 4, 100, 1000);
+            Config config = createClusterConfig(strategyClass, 100, 500, 500, null);
 
-        TopologyDetails topo1 = genTopology("topo-1", config, 8, 0, 2, 0, currentTime - 2, 10, "jerry");
-        TopologyDetails topo2 = genTopology("topo-2", config, 2, 0, 2, 0, currentTime - 2, 20, "jerry");
-        TopologyDetails topo3 = genTopology("topo-3", config, 1, 2, 1, 1, currentTime - 2, 20, "jerry");
+            TopologyDetails topo1 = genTopology("topo-1", config, 8, 0, 2, 0, currentTime - 2, 10, "jerry");
+            TopologyDetails topo2 = genTopology("topo-2", config, 2, 0, 2, 0, currentTime - 2, 20, "jerry");
+            TopologyDetails topo3 = genTopology("topo-3", config, 1, 2, 1, 1, currentTime - 2, 20, "jerry");
 
-        Topologies topologies = new Topologies(topo1, topo2, topo3);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
-        scheduler = new ResourceAwareScheduler();
+            Topologies topologies = new Topologies(topo1, topo2, topo3);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+            scheduler = new ResourceAwareScheduler();
 
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
 
-        assertFalse(cluster.getAssignmentById(topo1.getId()) != null, "Topo-1 unscheduled?");
-        assertTrue(cluster.getAssignmentById(topo2.getId()) != null, "Topo-2 scheduled?");
-        assertEquals(4, cluster.getAssignmentById(topo2.getId()).getExecutorToSlot().size(), "Topo-2 all executors scheduled?");
-        assertTrue(cluster.getAssignmentById(topo3.getId()) != null, "Topo-3 scheduled?");
-        assertEquals(3, cluster.getAssignmentById(topo3.getId()).getExecutorToSlot().size(), "Topo-3 all executors scheduled?");
+            assertFalse(cluster.getAssignmentById(topo1.getId()) != null, "Topo-1 unscheduled?");
+            assertTrue(cluster.getAssignmentById(topo2.getId()) != null, "Topo-2 scheduled?");
+            assertEquals(4, cluster.getAssignmentById(topo2.getId()).getExecutorToSlot().size(), "Topo-2 all executors scheduled?");
+            assertTrue(cluster.getAssignmentById(topo3.getId()) != null, "Topo-3 scheduled?");
+            assertEquals(3, cluster.getAssignmentById(topo3.getId()).getExecutorToSlot().size(), "Topo-3 all executors scheduled?");
+        }
     }
 
     /**
@@ -996,18 +1002,20 @@ public class TestResourceAwareScheduler {
      */
     @Test
     public void minCpuWorkerJustFits() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(1, 4, 100, 60000);
-        Config config = createClusterConfig(10, 500, 500, null);
-        config.put(DaemonConfig.STORM_WORKER_MIN_CPU_PCORE_PERCENT, 50.0);
-        TopologyDetails topo1 = genTopology("topo-1", config, 10, 0, 1, 1, currentTime - 2, 20, "jerry");
-        Topologies topologies = new Topologies(topo1);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-        assertFalse(cluster.needsSchedulingRas(topo1));
-        assertTrue(cluster.getAssignmentById(topo1.getId()) != null, "Topo-1 scheduled?");
+        for (Class strategyClass: strategyClasses) {
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(1, 4, 100, 60000);
+            Config config = createClusterConfig(strategyClass, 10, 500, 500, null);
+            config.put(DaemonConfig.STORM_WORKER_MIN_CPU_PCORE_PERCENT, 50.0);
+            TopologyDetails topo1 = genTopology("topo-1", config, 10, 0, 1, 1, currentTime - 2, 20, "jerry");
+            Topologies topologies = new Topologies(topo1);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+            assertFalse(cluster.needsSchedulingRas(topo1));
+            assertTrue(cluster.getAssignmentById(topo1.getId()) != null, "Topo-1 scheduled?");
+        }
     }
 
     /**
@@ -1016,89 +1024,76 @@ public class TestResourceAwareScheduler {
      */
     @Test
     public void minCpuPreventsThirdTopo() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(1, 4, 100, 60000);
-        Config config = createClusterConfig(10, 500, 500, null);
-        config.put(DaemonConfig.STORM_WORKER_MIN_CPU_PCORE_PERCENT, 40.0);
-        TopologyDetails topo1 = genTopology("topo-1", config, 2, 0, 1, 1, currentTime - 2, 20, "jerry");
-        TopologyDetails topo2 = genTopology("topo-2", config, 2, 0, 1, 1, currentTime - 2, 20, "jerry");
-        TopologyDetails topo3 = genTopology("topo-3", config, 2, 0, 1, 1, currentTime - 2, 20, "jerry");
-        Topologies topologies = new Topologies(topo1, topo2, topo3);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-        assertFalse(cluster.needsSchedulingRas(topo1));
-        assertFalse(cluster.needsSchedulingRas(topo2));
-        assertTrue(cluster.needsSchedulingRas(topo3));
-        assertTrue(cluster.getAssignmentById(topo1.getId()) != null, "topo-1 scheduled?");
-        assertTrue(cluster.getAssignmentById(topo2.getId()) != null, "topo-2 scheduled?");
-        assertFalse(cluster.getAssignmentById(topo3.getId()) != null, "topo-3 unscheduled?");
-
-        SchedulerAssignment assignment1 = cluster.getAssignmentById(topo1.getId());
-        assertEquals(1, assignment1.getSlots().size());
-        Map<WorkerSlot, WorkerResources> assignedSlots1 = assignment1.getScheduledResources();
-        double assignedCpu = 0.0;
-        for (Entry<WorkerSlot, WorkerResources> entry : assignedSlots1.entrySet()) {
-            WorkerResources wr = entry.getValue();
-            assignedCpu += wr.get_cpu();
-        }
-        assertEquals(40.0, assignedCpu, 0.001);
+        for (Class strategyClass: strategyClasses) {
+            if (strategyClass.getName().equals(RoundRobinResourceAwareStrategy.class.getName())) {
+                continue; // exclude RoundRbin from this test
+            }
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(1, 4, 100, 60000);
+            Config config = createClusterConfig(strategyClass, 10, 500, 500, null);
+            config.put(DaemonConfig.STORM_WORKER_MIN_CPU_PCORE_PERCENT, 40.0);
+            TopologyDetails topo1 = genTopology("topo-1", config, 2, 0, 1, 1, currentTime - 2, 20, "jerry");
+            TopologyDetails topo2 = genTopology("topo-2", config, 2, 0, 1, 1, currentTime - 2, 20, "jerry");
+            TopologyDetails topo3 = genTopology("topo-3", config, 2, 0, 1, 1, currentTime - 2, 20, "jerry");
+            Topologies topologies = new Topologies(topo1, topo2, topo3);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+            assertFalse(cluster.needsSchedulingRas(topo1), "using " + strategyClass);
+            assertFalse(cluster.needsSchedulingRas(topo2), "using " + strategyClass);
+            assertTrue(cluster.needsSchedulingRas(topo3), "using " + strategyClass);
+            assertTrue(cluster.getAssignmentById(topo1.getId()) != null, "topo-1 scheduled? using " + strategyClass);
+            assertTrue(cluster.getAssignmentById(topo2.getId()) != null, "topo-2 scheduled? using " + strategyClass);
+            assertFalse(cluster.getAssignmentById(topo3.getId()) != null, "topo-3 unscheduled? using " + strategyClass);
+
+            SchedulerAssignment assignment1 = cluster.getAssignmentById(topo1.getId());
+            assertEquals(1, assignment1.getSlots().size());
+            Map<WorkerSlot, WorkerResources> assignedSlots1 = assignment1.getScheduledResources();
+            double assignedCpu = 0.0;
+            for (Entry<WorkerSlot, WorkerResources> entry : assignedSlots1.entrySet()) {
+                WorkerResources wr = entry.getValue();
+                assignedCpu += wr.get_cpu();
+            }
+            assertEquals(40.0, assignedCpu, 0.001);
 
-        SchedulerAssignment assignment2 = cluster.getAssignmentById(topo2.getId());
-        assertEquals(1, assignment2.getSlots().size());
-        Map<WorkerSlot, WorkerResources> assignedSlots2 = assignment2.getScheduledResources();
-        assignedCpu = 0.0;
-        for (Entry<WorkerSlot, WorkerResources> entry : assignedSlots2.entrySet()) {
-            WorkerResources wr = entry.getValue();
-            assignedCpu += wr.get_cpu();
+            SchedulerAssignment assignment2 = cluster.getAssignmentById(topo2.getId());
+            assertEquals(1, assignment2.getSlots().size());
+            Map<WorkerSlot, WorkerResources> assignedSlots2 = assignment2.getScheduledResources();
+            assignedCpu = 0.0;
+            for (Entry<WorkerSlot, WorkerResources> entry : assignedSlots2.entrySet()) {
+                WorkerResources wr = entry.getValue();
+                assignedCpu += wr.get_cpu();
+            }
+            assertEquals(40.0, assignedCpu, 0.001);
         }
-        assertEquals(40.0, assignedCpu, 0.001);
     }
 
     @Test
     public void testMinCpuMaxMultipleSupervisors() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(3, 4, 300, 60000);
-        Config config = createClusterConfig(5, 50, 50, null);
-        config.put(DaemonConfig.STORM_WORKER_MIN_CPU_PCORE_PERCENT, 100.0);
-        TopologyDetails topo0 = genTopology("topo-0", config, 4, 5, 1, 1, currentTime - 2, 20, "jerry");
-        TopologyDetails topo1 = genTopology("topo-1", config, 4, 5, 1, 1, currentTime - 2, 20, "jerry");
-        TopologyDetails topo2 = genTopology("topo-2", config, 4, 5, 1, 1, currentTime - 2, 20, "jerry");
-        TopologyDetails topo3 = genTopology("topo-3", config, 4, 5, 1, 1, currentTime - 2, 20, "jerry");
-        TopologyDetails topo4 = genTopology("topo-4", config, 4, 5, 1, 1, currentTime - 2, 20, "jerry");
-        TopologyDetails topo5 = genTopology("topo-5", config, 4, 5, 1, 1, currentTime - 2, 20, "jerry");
-        TopologyDetails topo6 = genTopology("topo-6", config, 4, 5, 1, 1, currentTime - 2, 20, "jerry");
-        TopologyDetails topo7 = genTopology("topo-7", config, 4, 5, 1, 1, currentTime - 2, 20, "jerry");
-        TopologyDetails topo8 = genTopology("topo-8", config, 4, 5, 1, 1, currentTime - 2, 20, "jerry");
-        TopologyDetails topo9 = genTopology("topo-9", config, 4, 5, 1, 1, currentTime - 2, 20, "jerry");
-        Topologies topologies = new Topologies(topo0, topo1, topo2, topo3, topo4, topo5, topo6, topo7, topo8, topo9);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
+        int topoCnt = 10;
+        for (Class strategyClass: strategyClasses) {
+            if (strategyClass.getName().equals(RoundRobinResourceAwareStrategy.class.getName())) {
+                continue; // exclude RoundRbin from this test
+            }
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(3, 4, 300, 60000);
+            Config config = createClusterConfig(strategyClass, 5, 50, 50, null);
+            config.put(DaemonConfig.STORM_WORKER_MIN_CPU_PCORE_PERCENT, 100.0);
+            TopologyDetails[] topos = new TopologyDetails[topoCnt];
+            for (int i = 0 ; i < topoCnt ; i++) {
+                String topoName = "topo-" + i;
+                topos[i] = genTopology(topoName, config, 4, 5, 1, 1, currentTime - 2, 20, "jerry");
+            }
+            Topologies topologies = new Topologies(topos);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
 
-        assertFalse(cluster.needsSchedulingRas(topo0));
-        assertFalse(cluster.needsSchedulingRas(topo1));
-        assertFalse(cluster.needsSchedulingRas(topo2));
-        assertFalse(cluster.needsSchedulingRas(topo3));
-        assertFalse(cluster.needsSchedulingRas(topo4));
-        assertFalse(cluster.needsSchedulingRas(topo5));
-        assertFalse(cluster.needsSchedulingRas(topo6));
-        assertFalse(cluster.needsSchedulingRas(topo7));
-        assertFalse(cluster.needsSchedulingRas(topo8));
-        assertTrue(cluster.needsSchedulingRas(topo9));
-
-        assertTrue(cluster.getAssignmentById(topo0.getId()) != null,"topo-0 scheduled?");
-        assertTrue(cluster.getAssignmentById(topo1.getId()) != null, "topo-1 scheduled?");
-        assertTrue(cluster.getAssignmentById(topo2.getId()) != null, "topo-2 scheduled?");
-        assertTrue(cluster.getAssignmentById(topo3.getId()) != null, "topo-3 scheduled?");
-        assertTrue(cluster.getAssignmentById(topo4.getId()) != null, "topo-4 scheduled?");
-        assertTrue(cluster.getAssignmentById(topo5.getId()) != null, "topo-5 scheduled?");
-        assertTrue(cluster.getAssignmentById(topo6.getId()) != null, "topo-6 scheduled?");
-        assertTrue(cluster.getAssignmentById(topo7.getId()) != null, "topo-7 scheduled?");
-        assertTrue(cluster.getAssignmentById(topo8.getId()) != null, "topo-8 scheduled?");
-        assertFalse(cluster.getAssignmentById(topo9.getId()) != null, "topo-9 unscheduled?");
+            // topo-9 will not be scheduled
+            assertTopologiesFullyScheduled(cluster, strategyClass, topoCnt - 1);
+        }
     }
 
     /**
@@ -1107,19 +1102,21 @@ public class TestResourceAwareScheduler {
      */
     @Test
     public void minCpuWorkerSplitFails() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(1, 4, 100, 60000);
-        Config config = createClusterConfig(10, 500, 500, null);
-        config.put(DaemonConfig.STORM_WORKER_MIN_CPU_PCORE_PERCENT, 50.0);
-        TopologyDetails topo1 = genTopology("topo-1", config, 10, 0, 1, 1, currentTime - 2, 20,
-                "jerry", 2000.0);
-        Topologies topologies = new Topologies(topo1);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-        assertTrue(cluster.needsSchedulingRas(topo1));
-        assertFalse(cluster.getAssignmentById(topo1.getId()) != null, "Topo-1 unscheduled?");
+        for (Class strategyClass: strategyClasses) {
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(1, 4, 100, 60000);
+            Config config = createClusterConfig(strategyClass, 10, 500, 500, null);
+            config.put(DaemonConfig.STORM_WORKER_MIN_CPU_PCORE_PERCENT, 50.0);
+            TopologyDetails topo1 = genTopology("topo-1", config, 10, 0, 1, 1, currentTime - 2, 20,
+                    "jerry", 2000.0);
+            Topologies topologies = new Topologies(topo1);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+            assertTrue(cluster.needsSchedulingRas(topo1));
+            assertFalse(cluster.getAssignmentById(topo1.getId()) != null, "Topo-1 unscheduled?");
+        }
     }
 
     protected static class TimeBlockResult {
@@ -1199,23 +1196,23 @@ public class TestResourceAwareScheduler {
         GenericResourceAwareStrategy    7.78
         ConstraintSolverStrategy        7.75
         */
-
         final int numNodes = 500;
         final int numRuns = 5;
 
         Map<String, Config> strategyToConfigs = new HashMap<>();
-        strategyToConfigs.put(getDefaultResourceAwareStrategyClass().getName(), createClusterConfig(10, 10, 0, null));
-        strategyToConfigs.put(getGenericResourceAwareStrategyClass().getName(), createGrasClusterConfig(10, 10, 0, null, null));
-        strategyToConfigs.put(ConstraintSolverStrategy.class.getName(), createCSSClusterConfig(10, 10, 0, null));
-
-        Map<String, TimeBlockResult> strategyToTimeBlockResults = new HashMap<>();
-
         // AcceptedBlockTimeRatios obtained by empirical testing (see comment block above)
         Map<String, Double> strategyToAcceptedBlockTimeRatios = new HashMap<>();
-        strategyToAcceptedBlockTimeRatios.put(getDefaultResourceAwareStrategyClass().getName(), 6.96);
-        strategyToAcceptedBlockTimeRatios.put(getGenericResourceAwareStrategyClass().getName(), 7.78);
+        for (Class strategyClass: strategyClasses) {
+            strategyToConfigs.put(strategyClass.getName(), createClusterConfig(strategyClass, 10, 10, 0, null));
+            strategyToAcceptedBlockTimeRatios.put(strategyClass.getName(), 6.96);
+        }
+        strategyToAcceptedBlockTimeRatios.put(DefaultResourceAwareStrategy.class.getName(), 6.96);
+        strategyToAcceptedBlockTimeRatios.put(GenericResourceAwareStrategy.class.getName(), 7.78);
+        strategyToConfigs.put(ConstraintSolverStrategy.class.getName(), createCSSClusterConfig(10, 10, 0, null));
         strategyToAcceptedBlockTimeRatios.put(ConstraintSolverStrategy.class.getName(), 7.75);
 
+        Map<String, TimeBlockResult> strategyToTimeBlockResults = new HashMap<>();
+
         // Get first and last block times for multiple runs and strategies
         long startTime = Time.currentTimeMillis();
         for (Entry<String, Config> strategyConfig : strategyToConfigs.entrySet()) {
@@ -1339,73 +1336,87 @@ public class TestResourceAwareScheduler {
      */
     @Test
     public void testMultipleSpoutsAndCyclicTopologies() {
+        for (Class strategyClass: strategyClasses) {
+            String strategyClassName = strategyClass.getName();
+
+            TopologyBuilder builder = new TopologyBuilder();
+
+            builder.setSpout("spout-1", new TestSpout(),
+                    5);
+            builder.setSpout("spout-2", new TestSpout(),
+                    5);
+            builder.setBolt("bolt-1", new TestBolt(),
+                    5).shuffleGrouping("spout-1").shuffleGrouping("bolt-3");
+            builder.setBolt("bolt-2", new TestBolt(),
+                    5).shuffleGrouping("bolt-1");
+            builder.setBolt("bolt-3", new TestBolt(),
+                    5).shuffleGrouping("bolt-2").shuffleGrouping("spout-2");
+
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(25, 1, 100, 1000);
+            Config config = createClusterConfig(strategyClass, 100, 500, 500, null);
+
+            StormTopology stormTopology = builder.createTopology();
+            config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
+            TopologyDetails topo = new TopologyDetails("topo-1", config, stormTopology,
+                    0, genExecsAndComps(stormTopology), 0, "jerry");
+
+            Topologies topologies = new Topologies(topo);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
 
-        TopologyBuilder builder = new TopologyBuilder();
-
-        builder.setSpout("spout-1", new TestSpout(),
-                         5);
-        builder.setSpout("spout-2", new TestSpout(),
-                         5);
-        builder.setBolt("bolt-1", new TestBolt(),
-                        5).shuffleGrouping("spout-1").shuffleGrouping("bolt-3");
-        builder.setBolt("bolt-2", new TestBolt(),
-                        5).shuffleGrouping("bolt-1");
-        builder.setBolt("bolt-3", new TestBolt(),
-                        5).shuffleGrouping("bolt-2").shuffleGrouping("spout-2");
-
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(25, 1, 100, 1000);
-        Config config = createClusterConfig(100, 500, 500, null);
-
-        StormTopology stormTopology = builder.createTopology();
-        config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
-        TopologyDetails topo = new TopologyDetails("topo-1", config, stormTopology,
-                                                   0, genExecsAndComps(stormTopology), 0, "jerry");
-
-        Topologies topologies = new Topologies(topo);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-
-        assertTrue(cluster.getAssignmentById(topo.getId()) != null, "Topo scheduled?");
-        assertEquals(25, cluster.getAssignmentById(topo.getId()).getExecutorToSlot().size(), "Topo all executors scheduled?");
+            assertTrue(cluster.getAssignmentById(topo.getId()) != null, "Topo scheduled?");
+            assertEquals(25, cluster.getAssignmentById(topo.getId()).getExecutorToSlot().size(), "Topo all executors scheduled?");
+        }
     }
 
     @Test
     public void testSchedulerStrategyWhitelist() {
         Map<String, Object> config = ConfigUtils.readStormConfig();
-        String allowed = getDefaultResourceAwareStrategyClass().getName();
-        config.put(Config.NIMBUS_SCHEDULER_STRATEGY_CLASS_WHITELIST, Arrays.asList(allowed));
+        for (Class strategyClass: strategyClasses) {
+            String strategyClassName = strategyClass.getName();
+            String allowed = strategyClassName;
+            config.put(Config.NIMBUS_SCHEDULER_STRATEGY_CLASS_WHITELIST, Arrays.asList(allowed));
 
-        Object sched = ReflectionUtils.newSchedulerStrategyInstance(allowed, config);
-        assertEquals(sched.getClass().getName(), allowed);
+            Object sched = ReflectionUtils.newSchedulerStrategyInstance(allowed, config);
+            assertEquals(sched.getClass().getName(), allowed);
+        }
     }
 
     @Test
     public void testSchedulerStrategyWhitelistException() {
         Map<String, Object> config = ConfigUtils.readStormConfig();
         String allowed = "org.apache.storm.scheduler.resource.strategies.scheduling.SomeNonExistantStrategy";
-        String notAllowed = getDefaultResourceAwareStrategyClass().getName();
-        config.put(Config.NIMBUS_SCHEDULER_STRATEGY_CLASS_WHITELIST, Arrays.asList(allowed));
+        for (Class strategyClass: strategyClasses) {
+            String strategyClassName = strategyClass.getName();
+            String notAllowed = strategyClassName;
+            config.put(Config.NIMBUS_SCHEDULER_STRATEGY_CLASS_WHITELIST, Arrays.asList(allowed));
 
-        Assertions.assertThrows(DisallowedStrategyException.class, () -> ReflectionUtils.newSchedulerStrategyInstance(notAllowed, config));        
+            Assertions.assertThrows(DisallowedStrategyException.class, () -> ReflectionUtils.newSchedulerStrategyInstance(notAllowed, config));
+        }
     }
 
     @Test
     public void testSchedulerStrategyEmptyWhitelist() {
         Map<String, Object> config = ConfigUtils.readStormConfig();
-        String allowed = getDefaultResourceAwareStrategyClass().getName();
-
-        Object sched = ReflectionUtils.newSchedulerStrategyInstance(allowed, config);
-        assertEquals(sched.getClass().getName(), allowed);
+        for (Class strategyClass: strategyClasses) {
+            String strategyClassName = strategyClass.getName();
+            String allowed = strategyClassName;
+            Object sched = ReflectionUtils.newSchedulerStrategyInstance(allowed, config);
+            assertEquals(sched.getClass().getName(), allowed);
+        }
     }
 
     @PerformanceTest
     @Test
     public void testLargeTopologiesOnLargeClusters() {
-        Assertions.assertTimeoutPreemptively(Duration.ofSeconds(30), 
-            () -> testLargeTopologiesCommon(getDefaultResourceAwareStrategyClass().getName(), false, 1));
+        for (Class strategyClass: strategyClasses) {
+            String strategyClassName = strategyClass.getName();
+            Assertions.assertTimeoutPreemptively(Duration.ofSeconds(30),
+                    () -> testLargeTopologiesCommon(strategyClassName, false, 1));
+        }
         
     }
     
@@ -1413,7 +1424,14 @@ public class TestResourceAwareScheduler {
     @Test
     public void testLargeTopologiesOnLargeClustersGras() {
         Assertions.assertTimeoutPreemptively(Duration.ofSeconds(75),
-            () -> testLargeTopologiesCommon(getGenericResourceAwareStrategyClass().getName(), true, 1));
+                () -> testLargeTopologiesCommon(GenericResourceAwareStrategy.class.getName(), true, 1));
+    }
+
+    @PerformanceTest
+    @Test
+    public void testLargeTopologiesOnLargeClustersRoundRobin() {
+        Assertions.assertTimeoutPreemptively(Duration.ofSeconds(30),
+                () -> testLargeTopologiesCommon(RoundRobinResourceAwareStrategy.class.getName(), true, 1));
     }
 
     public static class NeverEndingSchedulingStrategy extends BaseResourceAwareStrategy {
@@ -1435,96 +1453,103 @@ public class TestResourceAwareScheduler {
 
     @Test
     public void testStrategyTakingTooLong() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(8, 4, 100, 1000);
-        Config config = createClusterConfig(100, 500, 500, null);
-        List<String> allowedSchedulerStrategies = new ArrayList<>();
-        allowedSchedulerStrategies.add(getDefaultResourceAwareStrategyClass().getName());
-        allowedSchedulerStrategies.add(DefaultResourceAwareStrategyOld.class.getName());
-        allowedSchedulerStrategies.add(NeverEndingSchedulingStrategy.class.getName());
-        config.put(Config.NIMBUS_SCHEDULER_STRATEGY_CLASS_WHITELIST, allowedSchedulerStrategies);
-        config.put(DaemonConfig.SCHEDULING_TIMEOUT_SECONDS_PER_TOPOLOGY, 30);
-
-        TopologyDetails topo1 = genTopology("topo-1", config, 1, 0, 2, 0, currentTime - 2, 10, "jerry");
-        TopologyDetails topo3 = genTopology("topo-3", config, 1, 2, 1, 1, currentTime - 2, 20, "jerry");
-
-        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, NeverEndingSchedulingStrategy.class.getName());
-        TopologyDetails topo2 = genTopology("topo-2", config, 2, 0, 2, 0, currentTime - 2, 20, "jerry");
-
-        Topologies topologies = new Topologies(topo1, topo2, topo3);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-        scheduler = new ResourceAwareScheduler();
+        for (Class strategyClass: strategyClasses) {
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(8, 4, 100, 1000);
+            Config config = createClusterConfig(strategyClass, 100, 500, 500, null);
+            List<String> allowedSchedulerStrategies = new ArrayList<>();
+            allowedSchedulerStrategies.add(DefaultResourceAwareStrategy.class.getName());
+            allowedSchedulerStrategies.add(DefaultResourceAwareStrategyOld.class.getName());
+            allowedSchedulerStrategies.add(GenericResourceAwareStrategy.class.getName());
+            allowedSchedulerStrategies.add(GenericResourceAwareStrategyOld.class.getName());
+            allowedSchedulerStrategies.add(RoundRobinResourceAwareStrategy.class.getName());
+            allowedSchedulerStrategies.add(NeverEndingSchedulingStrategy.class.getName());
+            config.put(Config.NIMBUS_SCHEDULER_STRATEGY_CLASS_WHITELIST, allowedSchedulerStrategies);
+            config.put(DaemonConfig.SCHEDULING_TIMEOUT_SECONDS_PER_TOPOLOGY, 30);
+
+            TopologyDetails topo1 = genTopology("topo-1", config, 1, 0, 2, 0, currentTime - 2, 10, "jerry");
+            TopologyDetails topo3 = genTopology("topo-3", config, 1, 2, 1, 1, currentTime - 2, 20, "jerry");
+
+            config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, NeverEndingSchedulingStrategy.class.getName());
+            TopologyDetails topo2 = genTopology("topo-2", config, 2, 0, 2, 0, currentTime - 2, 20, "jerry");
+
+            Topologies topologies = new Topologies(topo1, topo2, topo3);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+            scheduler = new ResourceAwareScheduler();
 
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
 
-        assertFalse(cluster.needsSchedulingRas(topo1));
-        assertTrue(cluster.needsSchedulingRas(topo2));
-        assertFalse(cluster.needsSchedulingRas(topo3));
-
-        assertTrue(cluster.getAssignmentById(topo1.getId()) != null, "Topo-1 scheduled?");
-        assertEquals(2, cluster.getAssignmentById(topo1.getId()).getExecutorToSlot().size(), "Topo-1 all executors scheduled?");
-        assertTrue(cluster.getAssignmentById(topo2.getId()) == null, "Topo-2 not scheduled");
-        assertEquals("Scheduling took too long for " + topo2.getId() + " using strategy "
-                + NeverEndingSchedulingStrategy.class.getName()
-                + " timeout after 30 seconds using config scheduling.timeout.seconds.per.topology.", cluster.getStatusMap().get(topo2.getId()));
-        assertTrue(cluster.getAssignmentById(topo3.getId()) != null, "Topo-3 scheduled?");
-        assertEquals(3, cluster.getAssignmentById(topo3.getId()).getExecutorToSlot().size(), "Topo-3 all executors scheduled?");
+            assertFalse(cluster.needsSchedulingRas(topo1));
+            assertTrue(cluster.needsSchedulingRas(topo2));
+            assertFalse(cluster.needsSchedulingRas(topo3));
+
+            assertTrue(cluster.getAssignmentById(topo1.getId()) != null, "Topo-1 scheduled?");
+            assertEquals(2, cluster.getAssignmentById(topo1.getId()).getExecutorToSlot().size(), "Topo-1 all executors scheduled?");
+            assertTrue(cluster.getAssignmentById(topo2.getId()) == null, "Topo-2 not scheduled");
+            assertEquals("Scheduling took too long for " + topo2.getId() + " using strategy "
+                    + NeverEndingSchedulingStrategy.class.getName()
+                    + " timeout after 30 seconds using config scheduling.timeout.seconds.per.topology.", cluster.getStatusMap().get(topo2.getId()));
+            assertTrue(cluster.getAssignmentById(topo3.getId()) != null, "Topo-3 scheduled?");
+            assertEquals(3, cluster.getAssignmentById(topo3.getId()).getExecutorToSlot().size(), "Topo-3 all executors scheduled?");
+        }
     }
 
     public void testLargeTopologiesCommon(final String strategy, final boolean includeGpu, final int multiplier) {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisorsWithRacks(25 * multiplier, 40, 66, 3 * multiplier, 0, 4700, 226200, new HashMap<>());
-        if (includeGpu) {
-            HashMap<String, Double> extraResources = new HashMap<>();
-            extraResources.put("my.gpu", 1.0);
-            supMap.putAll(genSupervisorsWithRacks(3 * multiplier, 40, 66, 0, 0, 4700, 226200, extraResources));
-        }
+        for (Class strategyClass: strategyClasses) {
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisorsWithRacks(25 * multiplier, 40, 66, 3 * multiplier, 0, 4700, 226200, new HashMap<>());
+            if (includeGpu) {
+                HashMap<String, Double> extraResources = new HashMap<>();
+                extraResources.put("my.gpu", 1.0);
+                supMap.putAll(genSupervisorsWithRacks(3 * multiplier, 40, 66, 0, 0, 4700, 226200, extraResources));
+            }
 
-        Config config = new Config();
-        config.putAll(createClusterConfig(88, 775, 25, null));
-        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, strategy);
+            Config config = new Config();
+            config.putAll(createClusterConfig(strategyClass, 88, 775, 25, null));
+            config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, strategy);
 
-        scheduler = new ResourceAwareScheduler();
+            scheduler = new ResourceAwareScheduler();
 
-        Map<String, TopologyDetails> topologyDetailsMap = new HashMap<>();
-        for (int i = 0; i < 11 * multiplier; i++) {
-            TopologyDetails td = genTopology(String.format("topology-%05d", i), config, 5,
-                40, 30, 114, 0, 0, "user", 8192);
-            topologyDetailsMap.put(td.getId(), td);
-        }
-        if (includeGpu) {
-            for (int i = 0; i < multiplier; i++) {
-                TopologyBuilder builder = topologyBuilder(5, 40, 30, 114);
-                builder.setBolt("gpu-bolt", new TestBolt(), 40)
-                    .addResource("my.gpu", 1.0)
-                    .shuffleGrouping("spout-0");
-                TopologyDetails td = topoToTopologyDetails(String.format("topology-gpu-%05d", i), config, builder.createTopology(), 0, 0,
-                    "user", 8192);
+            Map<String, TopologyDetails> topologyDetailsMap = new HashMap<>();
+            for (int i = 0; i < 11 * multiplier; i++) {
+                TopologyDetails td = genTopology(String.format("topology-%05d", i), config, 5,
+                        40, 30, 114, 0, 0, "user", 8192);
                 topologyDetailsMap.put(td.getId(), td);
             }
-        }
-        Topologies topologies = new Topologies(topologyDetailsMap);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+            if (includeGpu) {
+                for (int i = 0; i < multiplier; i++) {
+                    TopologyBuilder builder = topologyBuilder(5, 40, 30, 114);
+                    builder.setBolt("gpu-bolt", new TestBolt(), 40)
+                            .addResource("my.gpu", 1.0)
+                            .shuffleGrouping("spout-0");
+                    TopologyDetails td = topoToTopologyDetails(String.format("topology-gpu-%05d", i), config, builder.createTopology(), 0, 0,
+                            "user", 8192);
+                    topologyDetailsMap.put(td.getId(), td);
+                }
+            }
+            Topologies topologies = new Topologies(topologyDetailsMap);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
 
-        long startTime = Time.currentTimeMillis();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-        long schedulingDuration = Time.currentTimeMillis() - startTime;
-        LOG.info("Scheduling took " + schedulingDuration + " ms");
-        LOG.info("HAS {} SLOTS USED", cluster.getUsedSlots().size());
-
-        Map<String, SchedulerAssignment> assignments = new TreeMap<>(cluster.getAssignments());
-
-        for (Entry<String, SchedulerAssignment> entry: assignments.entrySet()) {
-            SchedulerAssignment sa = entry.getValue();
-            Map<String, AtomicLong> slotsPerRack = new TreeMap<>();
-            for (WorkerSlot slot : sa.getSlots()) {
-                String nodeId = slot.getNodeId();
-                String rack = supervisorIdToRackName(nodeId);
-                slotsPerRack.computeIfAbsent(rack, (r) -> new AtomicLong(0)).incrementAndGet();
+            long startTime = Time.currentTimeMillis();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+            long schedulingDuration = Time.currentTimeMillis() - startTime;
+            LOG.info("Scheduling took " + schedulingDuration + " ms");
+            LOG.info("HAS {} SLOTS USED", cluster.getUsedSlots().size());
+
+            Map<String, SchedulerAssignment> assignments = new TreeMap<>(cluster.getAssignments());
+
+            for (Entry<String, SchedulerAssignment> entry : assignments.entrySet()) {
+                SchedulerAssignment sa = entry.getValue();
+                Map<String, AtomicLong> slotsPerRack = new TreeMap<>();
+                for (WorkerSlot slot : sa.getSlots()) {
+                    String nodeId = slot.getNodeId();
+                    String rack = supervisorIdToRackName(nodeId);
+                    slotsPerRack.computeIfAbsent(rack, (r) -> new AtomicLong(0)).incrementAndGet();
+                }
+                LOG.info("{} => {}", entry.getKey(), slotsPerRack);
             }
-            LOG.info("{} => {}", entry.getKey(), slotsPerRack);
         }
     }
 
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestUser.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestUser.java
index 6071e4c6d..d3f989dd0 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestUser.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestUser.java
@@ -15,14 +15,20 @@ package org.apache.storm.scheduler.resource;
 import java.util.HashMap;
 import java.util.Map;
 import org.apache.storm.Config;
+import org.apache.storm.metric.StormMetricsRegistry;
 import org.apache.storm.scheduler.Cluster;
 import org.apache.storm.scheduler.INimbus;
 import org.apache.storm.scheduler.SupervisorDetails;
 import org.apache.storm.scheduler.Topologies;
 import org.apache.storm.scheduler.TopologyDetails;
 import org.apache.storm.scheduler.WorkerSlot;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
 import org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.INimbusTest;
 import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategyOld;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategyOld;
+import org.apache.storm.scheduler.resource.strategies.scheduling.RoundRobinResourceAwareStrategy;
 import org.apache.storm.utils.Time;
 import org.junit.jupiter.api.Test;
 
@@ -33,45 +39,48 @@ import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareSched
 import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.userResourcePool;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
-import org.apache.storm.metric.StormMetricsRegistry;
-import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
 
 public class TestUser {
+    private static final Class[] strategyClasses = {
+            DefaultResourceAwareStrategy.class,
+            DefaultResourceAwareStrategyOld.class,
+            RoundRobinResourceAwareStrategy.class,
+            GenericResourceAwareStrategy.class,
+            GenericResourceAwareStrategyOld.class,
+    };
 
-    protected Class getDefaultResourceAwareStrategyClass() {
-        return DefaultResourceAwareStrategy.class;
-    }
-
-    private Config createClusterConfig(double compPcore, double compOnHeap, double compOffHeap,
+    private Config createClusterConfig(Class strategyClass, double compPcore, double compOnHeap, double compOffHeap,
                                        Map<String, Map<String, Number>> pools) {
         Config config = TestUtilsForResourceAwareScheduler.createClusterConfig(compPcore, compOnHeap, compOffHeap, pools);
-        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, getDefaultResourceAwareStrategyClass().getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, strategyClass.getName());
         return config;
     }
 
     @Test
     public void testResourcePoolUtilization() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100, 1000);
-        double cpuGuarantee = 400.0;
-        double memoryGuarantee = 1000.0;
-        Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
-            userRes("user1", cpuGuarantee, memoryGuarantee));
-        Config config = createClusterConfig(100, 200, 200, resourceUserPool);
-        TopologyDetails topo1 = genTopology("topo-1", config, 1, 1, 2, 1, Time.currentTimeSecs() - 24, 9, "user1");
-        Topologies topologies = new Topologies(topo1);
+        for (Class strategyClass: strategyClasses) {
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100, 1000);
+            double cpuGuarantee = 400.0;
+            double memoryGuarantee = 1000.0;
+            Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
+                    userRes("user1", cpuGuarantee, memoryGuarantee));
+            Config config = createClusterConfig(strategyClass, 100, 200, 200, resourceUserPool);
+            TopologyDetails topo1 = genTopology("topo-1", config, 1, 1, 2, 1, Time.currentTimeSecs() - 24, 9, "user1");
+            Topologies topologies = new Topologies(topo1);
 
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-        User user1 = new User("user1", toDouble(resourceUserPool.get("user1")));
-        WorkerSlot slot = cluster.getAvailableSlots().get(0);
-        cluster.assign(slot, topo1.getId(), topo1.getExecutors());
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+            User user1 = new User("user1", toDouble(resourceUserPool.get("user1")));
+            WorkerSlot slot = cluster.getAvailableSlots().get(0);
+            cluster.assign(slot, topo1.getId(), topo1.getExecutors());
 
-        assertEquals(cpuGuarantee, user1.getCpuResourceGuaranteed(), 0.001, "check cpu resource guarantee");
-        assertEquals(memoryGuarantee, user1.getMemoryResourceGuaranteed(), 0.001, "check memory resource guarantee");
+            assertEquals(cpuGuarantee, user1.getCpuResourceGuaranteed(), 0.001, "check cpu resource guarantee");
+            assertEquals(memoryGuarantee, user1.getMemoryResourceGuaranteed(), 0.001, "check memory resource guarantee");
 
-        assertEquals(((100.0 * 3.0) / cpuGuarantee), user1.getCpuResourcePoolUtilization(cluster), 0.001,
-            "check cpu resource pool utilization");
-        assertEquals(((200.0 + 200.0) * 3.0) / memoryGuarantee, user1.getMemoryResourcePoolUtilization(cluster), 0.001,
-            "check memory resource pool utilization");
+            assertEquals(((100.0 * 3.0) / cpuGuarantee), user1.getCpuResourcePoolUtilization(cluster), 0.001,
+                    "check cpu resource pool utilization");
+            assertEquals(((200.0 + 200.0) * 3.0) / memoryGuarantee, user1.getMemoryResourcePoolUtilization(cluster), 0.001,
+                    "check memory resource pool utilization");
+        }
     }
 }
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
index dcffa99c9..68e486557 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
@@ -34,6 +34,7 @@ import org.apache.storm.scheduler.resource.strategies.priority.DefaultScheduling
 import org.apache.storm.scheduler.resource.strategies.scheduling.ConstraintSolverStrategy;
 import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy;
 import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.RoundRobinResourceAwareStrategy;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
@@ -63,6 +64,8 @@ import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
 public class TestUtilsForResourceAwareScheduler {
     private static final Logger LOG = LoggerFactory.getLogger(TestUtilsForResourceAwareScheduler.class);
 
@@ -107,6 +110,19 @@ public class TestUtilsForResourceAwareScheduler {
                                                 Map<String, Map<String, Number>> pools) {
         Config config = createClusterConfig(compPcore, compOnHeap, compOffHeap, pools);
         config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, ConstraintSolverStrategy.class.getName());
+        Map<String, Map<String,Object>> modifiedConstraints = new HashMap<>();
+        Map<String, Object> contraints = new HashMap<>();
+        contraints.put("maxNodeCoLocationCnt", 1);
+        modifiedConstraints.put("testSpout", contraints);
+        config.put(Config.TOPOLOGY_RAS_CONSTRAINTS, modifiedConstraints);
+        return config;
+    }
+
+    public static Config createRoundRobinClusterConfig(double compPcore, double compOnHeap, double compOffHeap,
+                                                 Map<String, Map<String, Number>> pools, Map<String, Double> genericResourceMap) {
+        Config config = createClusterConfig(compPcore, compOnHeap, compOffHeap, pools);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_MAP, genericResourceMap);
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, RoundRobinResourceAwareStrategy.class.getName());
         return config;
     }
 
@@ -461,53 +477,73 @@ public class TestUtilsForResourceAwareScheduler {
         return m.find();
     }
 
-    public static void assertTopologiesNotScheduled(Cluster cluster, String... topoNames) {
+    public static void assertTopologiesNotScheduled(Cluster cluster, Class strategyClass, String... topoNames) {
         Topologies topologies = cluster.getTopologies();
         for (String topoName : topoNames) {
             TopologyDetails td = topologies.getByName(topoName);
-            assert (td != null) : topoName;
+            String errMsg = "topology " + topoName + " using " + strategyClass.getName();
+            assert (td != null) : errMsg;
             String topoId = td.getId();
             String status = cluster.getStatus(topoId);
-            assert (status != null) : topoName;
-            assert (!isStatusSuccess(status)) : topoName;
-            assert (cluster.getAssignmentById(topoId) == null) : topoName;
-            assert (cluster.needsSchedulingRas(td)) : topoName;
+            assert (status != null) : errMsg;
+            assert (!isStatusSuccess(status)) : errMsg;
+            assert (cluster.getAssignmentById(topoId) == null) : errMsg;
+            assert (cluster.needsSchedulingRas(td)) : errMsg;
         }
     }
 
-    public static void assertTopologiesFullyScheduled(Cluster cluster, String... topoNames) {
+    public static void assertTopologiesFullyScheduled(Cluster cluster, Class strategyClass, String... topoNames) {
         Topologies topologies = cluster.getTopologies();
         for (String topoName : topoNames) {
             TopologyDetails td = topologies.getByName(topoName);
-            assert (td != null) : topoName;
+            String errMsg = "topology " + topoName + " using " + strategyClass.getName();
+            assert (td != null) : errMsg;
             String topoId = td.getId();
             assertStatusSuccess(cluster, topoId);
-            assert (cluster.getAssignmentById(topoId) != null) : topoName;
-            assert (cluster.needsSchedulingRas(td) == false) : topoName;
+            assert (cluster.getAssignmentById(topoId) != null): errMsg;
+            assert (cluster.needsSchedulingRas(td) == false): errMsg;
+        }
+    }
+
+    public static void assertTopologiesFullyScheduled(Cluster cluster, Class strategyClass, int expectedScheduledCnt) {
+        List<String> toposScheduled = new ArrayList<>();
+        for (TopologyDetails td: cluster.getTopologies()) {
+            String topoId = td.getId();
+            if (!isStatusSuccess(cluster.getStatus(topoId))
+                || cluster.getAssignmentById(topoId) == null
+                || cluster.needsSchedulingRas(td)) {
+                continue;
+            }
+            toposScheduled.add(td.getName());
         }
+        String errMsg = String.format("Only following topologies are scheduled: %s using %s",
+                String.join(",", toposScheduled), strategyClass.getName());
+        assertEquals(expectedScheduledCnt, toposScheduled.size(), errMsg);
     }
 
-    public static void assertTopologiesBeenEvicted(Cluster cluster, Set<String> evictedTopologies, String... topoNames) {
+    public static void assertTopologiesBeenEvicted(Cluster cluster, Class strategyClass, Set<String> evictedTopologies, String... topoNames) {
         Topologies topologies = cluster.getTopologies();
         LOG.info("Evicted topos: {}", evictedTopologies);
         assert (evictedTopologies != null);
         for (String topoName : topoNames) {
+            String errMsg = "topology " + topoName + " using " + strategyClass.getName();
             TopologyDetails td = topologies.getByName(topoName);
-            assert (td != null) : topoName;
+            assert (td != null) : errMsg;
             String topoId = td.getId();
-            assert (evictedTopologies.contains(topoId)) : topoName;
+            assert (evictedTopologies.contains(topoId)) : errMsg;
         }
     }
 
-    public static void assertTopologiesNotBeenEvicted(Cluster cluster, Set<String> evictedTopologies, String... topoNames) {
+    public static void assertTopologiesNotBeenEvicted(Cluster cluster, Class strategyClass, Set<String> evictedTopologies, String... topoNames) {
         Topologies topologies = cluster.getTopologies();
         LOG.info("Evicted topos: {}", evictedTopologies);
         assert (evictedTopologies != null);
         for (String topoName : topoNames) {
+            String errMsg = "topology " + topoName + " using " + strategyClass.getName();
             TopologyDetails td = topologies.getByName(topoName);
-            assert (td != null) : topoName;
+            assert (td != null) : errMsg;
             String topoId = td.getId();
-            assert (!evictedTopologies.contains(topoId)) : topoName;
+            assert (!evictedTopologies.contains(topoId)) : errMsg;
         }
     }
 
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/eviction/TestDefaultEvictionStrategy.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/eviction/TestDefaultEvictionStrategy.java
index 304043b50..1f0302d64 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/eviction/TestDefaultEvictionStrategy.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/eviction/TestDefaultEvictionStrategy.java
@@ -27,6 +27,10 @@ import org.apache.storm.scheduler.Topologies;
 import org.apache.storm.scheduler.resource.ResourceAwareScheduler;
 import org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler;
 import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategyOld;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategyOld;
+import org.apache.storm.scheduler.resource.strategies.scheduling.RoundRobinResourceAwareStrategy;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
@@ -42,17 +46,20 @@ import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
 
 public class TestDefaultEvictionStrategy {
     private static final Logger LOG = LoggerFactory.getLogger(TestDefaultEvictionStrategy.class);
+    private static final Class[] strategyClasses = {
+            DefaultResourceAwareStrategy.class,
+            DefaultResourceAwareStrategyOld.class,
+            RoundRobinResourceAwareStrategy.class,
+            GenericResourceAwareStrategy.class,
+            GenericResourceAwareStrategyOld.class,
+    };
     private int currentTime = 1450418597;
     private IScheduler scheduler = null;
 
-    protected Class getDefaultResourceAwareStrategyClass() {
-        return DefaultResourceAwareStrategy.class;
-    }
-
-    private Config createClusterConfig(double compPcore, double compOnHeap, double compOffHeap,
+    private Config createClusterConfig(Class strategyClass, double compPcore, double compOnHeap, double compOffHeap,
                                        Map<String, Map<String, Number>> pools) {
         Config config = TestUtilsForResourceAwareScheduler.createClusterConfig(compPcore, compOnHeap, compOffHeap, pools);
-        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, getDefaultResourceAwareStrategyClass().getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, strategyClass.getName());
         return config;
     }
 
@@ -71,120 +78,126 @@ public class TestDefaultEvictionStrategy {
      */
     @Test
     public void testEviction() {
-      INimbus iNimbus = new INimbusTest();
-      Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100, 1000);
-        Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
-            userRes("jerry", 200, 2000),
-            userRes("bobby", 100, 1000),
-            userRes("derek", 200, 2000));
-        Config config = createClusterConfig(100, 500, 500, resourceUserPool);
-        Topologies topologies = new Topologies(
-            genTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 10, "jerry"),
-            genTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
-            genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20, "bobby"),
-            genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"));
-
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-        
-        assertTopologiesFullyScheduled(cluster, "topo-1", "topo-2", "topo-3", "topo-4");
-
-        //user jerry submits another topology
-        topologies = addTopologies(topologies,
-            genTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"));
-        cluster = new Cluster(cluster, topologies);
-        scheduler.schedule(topologies, cluster);
-        
-        //topo-3 evicted (lowest priority)
-        assertTopologiesFullyScheduled(cluster, "topo-1", "topo-2", "topo-4", "topo-6");
-        assertTopologiesNotScheduled(cluster, "topo-3");
+        for (Class strategyClass: strategyClasses) {
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100, 1000);
+            Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
+                    userRes("jerry", 200, 2000),
+                    userRes("bobby", 100, 1000),
+                    userRes("derek", 200, 2000));
+            Config config = createClusterConfig(strategyClass, 100, 500, 500, resourceUserPool);
+            Topologies topologies = new Topologies(
+                    genTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 10, "jerry"),
+                    genTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
+                    genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20, "bobby"),
+                    genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"));
+
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+
+            assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1", "topo-2", "topo-3", "topo-4");
+
+            //user jerry submits another topology
+            topologies = addTopologies(topologies,
+                    genTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"));
+            cluster = new Cluster(cluster, topologies);
+            scheduler.schedule(topologies, cluster);
+
+            //topo-3 evicted (lowest priority)
+            assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1", "topo-2", "topo-4", "topo-6");
+            assertTopologiesNotScheduled(cluster, strategyClass, "topo-3");
+        }
     }
 
     @Test
     public void testEvictMultipleTopologies() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100, 1000);
-        Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
-            userRes("jerry", 200, 2000),
-            userRes("derek", 100, 1000));
-        Config config = createClusterConfig(100, 500, 500, resourceUserPool);
-
-        Topologies topologies = new Topologies(
-            genTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
-            genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20, "bobby"),
-            genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"),
-            genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"));
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        LOG.info("\n\n\t\tScheduling topos 2 to 5...");
-        scheduler.schedule(topologies, cluster);
-        LOG.info("\n\n\t\tDone scheduling...");
-        assertTopologiesFullyScheduled(cluster, "topo-2", "topo-3", "topo-4", "topo-5");
-
-        //user jerry submits another topology
-        topologies = addTopologies(topologies,
-            genTopology("topo-1", config, 2, 0, 1, 0, currentTime - 2, 10, "jerry"));
-        cluster = new Cluster(cluster, topologies);
-        LOG.info("\n\n\t\tScheduling topos 1 to 5");
-        scheduler.schedule(topologies, cluster);
-        LOG.info("\n\n\t\tDone scheduling...");
-        //bobby has no guarantee so topo-2 and topo-3 evicted
-        assertTopologiesFullyScheduled(cluster, "topo-1", "topo-4", "topo-5");
-        assertTopologiesNotScheduled(cluster, "topo-2", "topo-3");
+        for (Class strategyClass: strategyClasses) {
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100, 1000);
+            Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
+                    userRes("jerry", 200, 2000),
+                    userRes("derek", 100, 1000));
+            Config config = createClusterConfig(strategyClass, 100, 500, 500, resourceUserPool);
+
+            Topologies topologies = new Topologies(
+                    genTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
+                    genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20, "bobby"),
+                    genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"),
+                    genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"));
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            LOG.info("\n\n\t\tScheduling topos 2 to 5...");
+            scheduler.schedule(topologies, cluster);
+            LOG.info("\n\n\t\tDone scheduling...");
+            assertTopologiesFullyScheduled(cluster, strategyClass,"topo-2", "topo-3", "topo-4", "topo-5");
+
+            //user jerry submits another topology
+            topologies = addTopologies(topologies,
+                    genTopology("topo-1", config, 2, 0, 1, 0, currentTime - 2, 10, "jerry"));
+            cluster = new Cluster(cluster, topologies);
+            LOG.info("\n\n\t\tScheduling topos 1 to 5");
+            scheduler.schedule(topologies, cluster);
+            LOG.info("\n\n\t\tDone scheduling...");
+            //bobby has no guarantee so topo-2 and topo-3 evicted
+            assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1", "topo-4", "topo-5");
+            assertTopologiesNotScheduled(cluster, strategyClass,"topo-2", "topo-3");
+        }
     }
 
     @Test
     public void testEvictMultipleTopologiesFromMultipleUsersInCorrectOrder() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100, 1000);
-        Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
-            userRes("jerry", 300, 3000),
-            userRes("derek", 100, 1000));
-        Config config = createClusterConfig(100, 500, 500, resourceUserPool);
-
-        Topologies topologies = new Topologies(
-            genTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
-            genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20, "bobby"),
-            genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"),
-            genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 15, 29, "derek"));
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-        
-        assertTopologiesFullyScheduled(cluster, "topo-2", "topo-3", "topo-4", "topo-5");
-
-        //user jerry submits another topology
-        topologies = addTopologies(topologies,
-            genTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 10, "jerry"));
-        cluster = new Cluster(cluster, topologies);
-        scheduler.schedule(topologies, cluster);
-
-        //topo-3 evicted since user bobby don't have any resource guarantees and topo-3 is the lowest priority for user bobby
-        assertTopologiesFullyScheduled(cluster, "topo-1", "topo-2", "topo-4", "topo-5");
-        assertTopologiesNotScheduled(cluster, "topo-3");
-        
-        topologies = addTopologies(topologies, 
-            genTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 10, "jerry"));
-        cluster = new Cluster(cluster, topologies);
-        scheduler.schedule(topologies, cluster);
-        
-        //topo-2 evicted since user bobby don't have any resource guarantees and topo-2 is the next lowest priority for user bobby
-        assertTopologiesFullyScheduled(cluster, "topo-1", "topo-4", "topo-5");
-        assertTopologiesNotScheduled(cluster, "topo-2", "topo-3");
-
-        topologies = addTopologies(topologies,
-            genTopology("topo-7", config, 1, 0, 1, 0, currentTime - 2, 10, "jerry"));
-        cluster = new Cluster(cluster, topologies);
-        scheduler.schedule(topologies, cluster);
-
-        // since user derek has exceeded his resource guarantee while user jerry has not topo-5 or topo-4 could be evicted because they have the same priority
-        // but topo-4 was submitted earlier thus we choose that one to evict (somewhat arbitrary)
-        assertTopologiesFullyScheduled(cluster, "topo-1", "topo-5", "topo-7");
-        assertTopologiesNotScheduled(cluster, "topo-2", "topo-3", "topo-4");
+        for (Class strategyClass: strategyClasses) {
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100, 1000);
+            Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
+                    userRes("jerry", 300, 3000),
+                    userRes("derek", 100, 1000));
+            Config config = createClusterConfig(strategyClass, 100, 500, 500, resourceUserPool);
+
+            Topologies topologies = new Topologies(
+                    genTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
+                    genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20, "bobby"),
+                    genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"),
+                    genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 15, 29, "derek"));
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+
+            assertTopologiesFullyScheduled(cluster, strategyClass, "topo-2", "topo-3", "topo-4", "topo-5");
+
+            //user jerry submits another topology
+            topologies = addTopologies(topologies,
+                    genTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 10, "jerry"));
+            cluster = new Cluster(cluster, topologies);
+            scheduler.schedule(topologies, cluster);
+
+            //topo-3 evicted since user bobby don't have any resource guarantees and topo-3 is the lowest priority for user bobby
+            assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1", "topo-2", "topo-4", "topo-5");
+            assertTopologiesNotScheduled(cluster, strategyClass,"topo-3");
+
+            topologies = addTopologies(topologies,
+                    genTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 10, "jerry"));
+            cluster = new Cluster(cluster, topologies);
+            scheduler.schedule(topologies, cluster);
+
+            //topo-2 evicted since user bobby don't have any resource guarantees and topo-2 is the next lowest priority for user bobby
+            assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1", "topo-4", "topo-5");
+            assertTopologiesNotScheduled(cluster, strategyClass, "topo-2", "topo-3");
+
+            topologies = addTopologies(topologies,
+                    genTopology("topo-7", config, 1, 0, 1, 0, currentTime - 2, 10, "jerry"));
+            cluster = new Cluster(cluster, topologies);
+            scheduler.schedule(topologies, cluster);
+
+            // since user derek has exceeded his resource guarantee while user jerry has not topo-5 or topo-4 could be evicted because they have the same priority
+            // but topo-4 was submitted earlier thus we choose that one to evict (somewhat arbitrary)
+            assertTopologiesFullyScheduled(cluster, strategyClass,"topo-1", "topo-5", "topo-7");
+            assertTopologiesNotScheduled(cluster, strategyClass,"topo-2", "topo-3", "topo-4");
+        }
     }
 
     /**
@@ -193,51 +206,53 @@ public class TestDefaultEvictionStrategy {
      */
     @Test
     public void testEvictTopologyFromItself() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100, 1000);
-        Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
-            userRes("jerry", 200, 2000),
-            userRes("bobby", 100, 1000),
-            userRes("derek", 100, 1000));
-        Config config = createClusterConfig(100, 500, 500, resourceUserPool);
-
-        Topologies topologies = new Topologies(
-            genTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"),
-            genTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"),
-            genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
-            genTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"));
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        LOG.info("\n\n\t\tScheduling topos 1,2,5,6");
-        scheduler.schedule(topologies, cluster);
-        LOG.info("\n\n\t\tDone Scheduling...");
-        
-        assertTopologiesFullyScheduled(cluster, "topo-1", "topo-2", "topo-5", "topo-6");
-
-        //user jerry submits another topology into a full cluster
-        // topo3 should not be able to scheduled
-        topologies = addTopologies(topologies,
-            genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 29, "jerry"));
-        cluster = new Cluster(cluster, topologies);
-        LOG.info("\n\n\t\tScheduling topos 1,2,3,5,6");
-        scheduler.schedule(topologies, cluster);
-        LOG.info("\n\n\t\tDone Scheduling...");
-
-        assertTopologiesFullyScheduled(cluster, "topo-1", "topo-2", "topo-5", "topo-6");
-        assertTopologiesNotScheduled(cluster, "topo-3");
-
-        //user jerry submits another topology but this one should be scheduled since it has higher priority than the
-        //rest of jerry's running topologies
-        topologies = addTopologies(topologies,
-            genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 10, "jerry"));
-        cluster = new Cluster(cluster, topologies);
-        LOG.info("\n\n\t\tScheduling topos 1-6");
-        scheduler.schedule(topologies, cluster);
-        LOG.info("\n\n\t\tDone Scheduling...");
-        
-        assertTopologiesFullyScheduled(cluster, "topo-1", "topo-4", "topo-5", "topo-6");
-        assertTopologiesNotScheduled(cluster, "topo-2", "topo-3");
+        for (Class strategyClass: strategyClasses) {
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100, 1000);
+            Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
+                    userRes("jerry", 200, 2000),
+                    userRes("bobby", 100, 1000),
+                    userRes("derek", 100, 1000));
+            Config config = createClusterConfig(strategyClass, 100, 500, 500, resourceUserPool);
+
+            Topologies topologies = new Topologies(
+                    genTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"),
+                    genTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"),
+                    genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
+                    genTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"));
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            LOG.info("\n\n\t\tScheduling topos 1,2,5,6");
+            scheduler.schedule(topologies, cluster);
+            LOG.info("\n\n\t\tDone Scheduling...");
+
+            assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1", "topo-2", "topo-5", "topo-6");
+
+            //user jerry submits another topology into a full cluster
+            // topo3 should not be able to scheduled
+            topologies = addTopologies(topologies,
+                    genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 29, "jerry"));
+            cluster = new Cluster(cluster, topologies);
+            LOG.info("\n\n\t\tScheduling topos 1,2,3,5,6");
+            scheduler.schedule(topologies, cluster);
+            LOG.info("\n\n\t\tDone Scheduling...");
+
+            assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1", "topo-2", "topo-5", "topo-6");
+            assertTopologiesNotScheduled(cluster, strategyClass, "topo-3");
+
+            //user jerry submits another topology but this one should be scheduled since it has higher priority than the
+            //rest of jerry's running topologies
+            topologies = addTopologies(topologies,
+                    genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 10, "jerry"));
+            cluster = new Cluster(cluster, topologies);
+            LOG.info("\n\n\t\tScheduling topos 1-6");
+            scheduler.schedule(topologies, cluster);
+            LOG.info("\n\n\t\tDone Scheduling...");
+
+            assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1", "topo-4", "topo-5", "topo-6");
+            assertTopologiesNotScheduled(cluster, strategyClass, "topo-2", "topo-3");
+        }
     }
 
     /**
@@ -245,50 +260,52 @@ public class TestDefaultEvictionStrategy {
      */
     @Test
     public void testOverGuaranteeEviction() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100, 1000);
-        Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
-            userRes("jerry", 70, 700),
-            userRes("bobby", 100, 1000),
-            userRes("derek", 25, 250));
-        Config config = createClusterConfig(100, 500, 500, resourceUserPool);
-
-        Topologies topologies = new Topologies(
-            genTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"),
-            genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
-            genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
-            genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"));
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        LOG.info("\n\n\t\tScheduling topos 1,3,4,5");
-        scheduler.schedule(topologies, cluster);
-        LOG.info("\n\n\t\tDone scheduling...");
-        
-        assertTopologiesFullyScheduled(cluster, "topo-1", "topo-3", "topo-4", "topo-5");
-
-        //user derek submits another topology into a full cluster
-        //topo6 should not be able to scheduled initially, but since topo6 has higher priority than topo5
-        //topo5 will be evicted so that topo6 can be scheduled
-        topologies = addTopologies(topologies,
-            genTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 10, "derek"));
-        cluster = new Cluster(cluster, topologies);
-        LOG.info("\n\n\t\tScheduling topos 1,3,4,5,6");
-        scheduler.schedule(topologies, cluster);
-        LOG.info("\n\n\t\tDone scheduling...");
-
-        assertTopologiesFullyScheduled(cluster, "topo-1", "topo-3", "topo-4", "topo-6");
-        assertTopologiesNotScheduled(cluster, "topo-5");
-
-        //user jerry submits topo2
-        topologies = addTopologies(topologies,
-            genTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"));
-        cluster = new Cluster(cluster, topologies);
-        LOG.info("\n\n\t\tScheduling topos 1-6");
-        scheduler.schedule(topologies, cluster);
-        LOG.info("\n\n\t\tDone scheduling...");
-
-        assertTopologiesFullyScheduled(cluster, "topo-1", "topo-3", "topo-4", "topo-6");
-        assertTopologiesNotScheduled(cluster, "topo-2", "topo-5");
+        for (Class strategyClass: strategyClasses) {
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100, 1000);
+            Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
+                    userRes("jerry", 70, 700),
+                    userRes("bobby", 100, 1000),
+                    userRes("derek", 25, 250));
+            Config config = createClusterConfig(strategyClass, 100, 500, 500, resourceUserPool);
+
+            Topologies topologies = new Topologies(
+                    genTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"),
+                    genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
+                    genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
+                    genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"));
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            LOG.info("\n\n\t\tScheduling topos 1,3,4,5");
+            scheduler.schedule(topologies, cluster);
+            LOG.info("\n\n\t\tDone scheduling...");
+
+            assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1", "topo-3", "topo-4", "topo-5");
+
+            //user derek submits another topology into a full cluster
+            //topo6 should not be able to scheduled initially, but since topo6 has higher priority than topo5
+            //topo5 will be evicted so that topo6 can be scheduled
+            topologies = addTopologies(topologies,
+                    genTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 10, "derek"));
+            cluster = new Cluster(cluster, topologies);
+            LOG.info("\n\n\t\tScheduling topos 1,3,4,5,6");
+            scheduler.schedule(topologies, cluster);
+            LOG.info("\n\n\t\tDone scheduling...");
+
+            assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1", "topo-3", "topo-4", "topo-6");
+            assertTopologiesNotScheduled(cluster, strategyClass, "topo-5");
+
+            //user jerry submits topo2
+            topologies = addTopologies(topologies,
+                    genTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"));
+            cluster = new Cluster(cluster, topologies);
+            LOG.info("\n\n\t\tScheduling topos 1-6");
+            scheduler.schedule(topologies, cluster);
+            LOG.info("\n\n\t\tDone scheduling...");
+
+            assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1", "topo-3", "topo-4", "topo-6");
+            assertTopologiesNotScheduled(cluster, strategyClass,"topo-2", "topo-5");
+        }
     }
 }
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/priority/TestFIFOSchedulingPriorityStrategy.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/priority/TestFIFOSchedulingPriorityStrategy.java
index c479f90bb..7977cfaff 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/priority/TestFIFOSchedulingPriorityStrategy.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/priority/TestFIFOSchedulingPriorityStrategy.java
@@ -27,6 +27,10 @@ import org.apache.storm.scheduler.Topologies;
 import org.apache.storm.scheduler.resource.ResourceAwareScheduler;
 import org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler;
 import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategyOld;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategyOld;
+import org.apache.storm.scheduler.resource.strategies.scheduling.RoundRobinResourceAwareStrategy;
 import org.apache.storm.utils.Time;
 import org.junit.jupiter.api.Test;
 
@@ -42,67 +46,72 @@ import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
 
 public class TestFIFOSchedulingPriorityStrategy {
     private static final Logger LOG = LoggerFactory.getLogger(TestFIFOSchedulingPriorityStrategy.class);
-
-    protected Class getDefaultResourceAwareStrategyClass() {
-        return DefaultResourceAwareStrategy.class;
-    }
-
-    private Config createClusterConfig(double compPcore, double compOnHeap, double compOffHeap,
+    private static final Class[] strategyClasses = {
+            DefaultResourceAwareStrategy.class,
+            DefaultResourceAwareStrategyOld.class,
+            RoundRobinResourceAwareStrategy.class,
+            GenericResourceAwareStrategy.class,
+            GenericResourceAwareStrategyOld.class,
+    };
+
+    private Config createClusterConfig(Class strategyClass, double compPcore, double compOnHeap, double compOffHeap,
                                              Map<String, Map<String, Number>> pools) {
         Config config = TestUtilsForResourceAwareScheduler.createClusterConfig(compPcore, compOnHeap, compOffHeap, pools);
-        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, getDefaultResourceAwareStrategyClass().getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, strategyClass.getName());
         return config;
     }
 
     @Test
     public void testFIFOEvictionStrategy() {
-        try (Time.SimulatedTime sim = new Time.SimulatedTime()) {
-            INimbus iNimbus = new INimbusTest();
-            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100.0, 1000.0);
-            Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
-                userRes("jerry", 200.0, 2000.0));
-            Config config = createClusterConfig(100, 500, 500, resourceUserPool);
-            config.put(DaemonConfig.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, FIFOSchedulingPriorityStrategy.class.getName());
-
-            Topologies topologies = new Topologies(
-                genTopology("topo-1-jerry", config, 1, 0, 1, 0, Time.currentTimeSecs() - 250, 20, "jerry"),
-                genTopology("topo-2-bobby", config, 1, 0, 1, 0, Time.currentTimeSecs() - 200, 10, "bobby"),
-                genTopology("topo-3-bobby", config, 1, 0, 1, 0, Time.currentTimeSecs() - 300, 20, "bobby"),
-                genTopology("topo-4-derek", config, 1, 0, 1, 0, Time.currentTimeSecs() - 201, 29, "derek"));
-            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-
-            ResourceAwareScheduler rs = new ResourceAwareScheduler();
-            rs.prepare(config, new StormMetricsRegistry());
-            try {
-                rs.schedule(topologies, cluster);
-
-                assertTopologiesFullyScheduled(cluster, "topo-1-jerry", "topo-2-bobby", "topo-3-bobby", "topo-4-derek");
-
-                LOG.info("\n\n\t\tINSERTING topo-5");
-                //new topology needs to be scheduled
-                //topo-3 should be evicted since it's been up the longest
-                topologies = addTopologies(topologies,
-                    genTopology("topo-5-derek", config, 1, 0, 1, 0, Time.currentTimeSecs() - 15, 29, "derek"));
-
-                cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-                rs.schedule(topologies, cluster);
-
-                assertTopologiesFullyScheduled(cluster, "topo-1-jerry", "topo-2-bobby", "topo-4-derek", "topo-5-derek");
-                assertTopologiesNotScheduled(cluster, "topo-3-bobby");
-
-                LOG.info("\n\n\t\tINSERTING topo-6");
-                //new topology needs to be scheduled.  topo-4 should be evicted. Even though topo-1 from user jerry is older, topo-1 will not be evicted
-                //since user jerry has enough resource guarantee
-                topologies = addTopologies(topologies,
-                    genTopology("topo-6-bobby", config, 1, 0, 1, 0, Time.currentTimeSecs() - 10, 29, "bobby"));
-
-                cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-                rs.schedule(topologies, cluster);
-
-                assertTopologiesFullyScheduled(cluster, "topo-1-jerry", "topo-2-bobby", "topo-5-derek", "topo-6-bobby");
-                assertTopologiesNotScheduled(cluster, "topo-3-bobby", "topo-4-derek");
-            } finally {
-                rs.cleanup();
+        for (Class strategyClass: strategyClasses) {
+            try (Time.SimulatedTime sim = new Time.SimulatedTime()) {
+                INimbus iNimbus = new INimbusTest();
+                Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 100.0, 1000.0);
+                Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
+                        userRes("jerry", 200.0, 2000.0));
+                Config config = createClusterConfig(strategyClass, 100, 500, 500, resourceUserPool);
+                config.put(DaemonConfig.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, FIFOSchedulingPriorityStrategy.class.getName());
+
+                Topologies topologies = new Topologies(
+                        genTopology("topo-1-jerry", config, 1, 0, 1, 0, Time.currentTimeSecs() - 250, 20, "jerry"),
+                        genTopology("topo-2-bobby", config, 1, 0, 1, 0, Time.currentTimeSecs() - 200, 10, "bobby"),
+                        genTopology("topo-3-bobby", config, 1, 0, 1, 0, Time.currentTimeSecs() - 300, 20, "bobby"),
+                        genTopology("topo-4-derek", config, 1, 0, 1, 0, Time.currentTimeSecs() - 201, 29, "derek"));
+                Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+
+                ResourceAwareScheduler rs = new ResourceAwareScheduler();
+                rs.prepare(config, new StormMetricsRegistry());
+                try {
+                    rs.schedule(topologies, cluster);
+
+                    assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1-jerry", "topo-2-bobby", "topo-3-bobby", "topo-4-derek");
+
+                    LOG.info("\n\n\t\tINSERTING topo-5");
+                    //new topology needs to be scheduled
+                    //topo-3 should be evicted since it's been up the longest
+                    topologies = addTopologies(topologies,
+                            genTopology("topo-5-derek", config, 1, 0, 1, 0, Time.currentTimeSecs() - 15, 29, "derek"));
+
+                    cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+                    rs.schedule(topologies, cluster);
+
+                    assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1-jerry", "topo-2-bobby", "topo-4-derek", "topo-5-derek");
+                    assertTopologiesNotScheduled(cluster, strategyClass, "topo-3-bobby");
+
+                    LOG.info("\n\n\t\tINSERTING topo-6");
+                    //new topology needs to be scheduled.  topo-4 should be evicted. Even though topo-1 from user jerry is older, topo-1 will not be evicted
+                    //since user jerry has enough resource guarantee
+                    topologies = addTopologies(topologies,
+                            genTopology("topo-6-bobby", config, 1, 0, 1, 0, Time.currentTimeSecs() - 10, 29, "bobby"));
+
+                    cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+                    rs.schedule(topologies, cluster);
+
+                    assertTopologiesFullyScheduled(cluster, strategyClass, "topo-1-jerry", "topo-2-bobby", "topo-5-derek", "topo-6-bobby");
+                    assertTopologiesNotScheduled(cluster, strategyClass, "topo-3-bobby", "topo-4-derek");
+                } finally {
+                    rs.cleanup();
+                }
             }
         }
     }
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/priority/TestGenericResourceAwareSchedulingPriorityStrategy.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/priority/TestGenericResourceAwareSchedulingPriorityStrategy.java
index 4d9b1782c..74cc5247d 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/priority/TestGenericResourceAwareSchedulingPriorityStrategy.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/priority/TestGenericResourceAwareSchedulingPriorityStrategy.java
@@ -29,7 +29,11 @@ import org.apache.storm.scheduler.Topologies;
 import org.apache.storm.scheduler.resource.ResourceAwareScheduler;
 import org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler;
 import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
+import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategyOld;
 import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategyOld;
+import org.apache.storm.scheduler.resource.strategies.scheduling.RoundRobinResourceAwareStrategy;
 import org.apache.storm.utils.Time;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.Test;
@@ -51,6 +55,10 @@ import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareSched
 import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.userResourcePool;
 
 public class TestGenericResourceAwareSchedulingPriorityStrategy {
+    private static final Class[] strategyClasses = {
+            GenericResourceAwareStrategy.class,
+            GenericResourceAwareStrategyOld.class,
+    };
     private final int currentTime = Time.currentTimeSecs();
     private IScheduler scheduler = null;
 
@@ -62,14 +70,10 @@ public class TestGenericResourceAwareSchedulingPriorityStrategy {
         }
     }
 
-    protected Class getGenericResourceAwareStrategyClass() {
-        return GenericResourceAwareStrategy.class;
-    }
-
-    private Config createGrasClusterConfig(double compPcore, double compOnHeap, double compOffHeap,
+    private Config createGrasClusterConfig(Class strategyClass, double compPcore, double compOnHeap, double compOffHeap,
                                                  Map<String, Map<String, Number>> pools, Map<String, Double> genericResourceMap) {
         Config config = TestUtilsForResourceAwareScheduler.createGrasClusterConfig(compPcore, compOnHeap, compOffHeap, pools, genericResourceMap);
-        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, getGenericResourceAwareStrategyClass().getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, strategyClass.getName());
         return config;
     }
 
@@ -81,36 +85,38 @@ public class TestGenericResourceAwareSchedulingPriorityStrategy {
     */
     @Test
     public void testDefaultSchedulingPriorityStrategyNotEvicting() {
-        Map<String, Double> requestedgenericResourcesMap = new HashMap<>();
-        requestedgenericResourcesMap.put("generic.resource.1", 40.0);
-        // Use full memory and cpu of the cluster capacity
-        Config ruiConf = createGrasClusterConfig(20, 50, 50, null, requestedgenericResourcesMap);
-        Config ethanConf = createGrasClusterConfig(80, 400, 500, null, Collections.emptyMap());
-        Topologies topologies = new Topologies(
-            genTopology("ethan-topo-1", ethanConf, 1, 0, 1, 0, currentTime - 2, 10, "ethan"),
-            genTopology("ethan-topo-2", ethanConf, 1, 0, 1, 0, currentTime - 2, 20, "ethan"),
-            genTopology("ethan-topo-3", ethanConf, 1, 0, 1, 0, currentTime - 2, 28, "ethan"),
-            genTopology("ethan-topo-4", ethanConf, 1, 0, 1, 0, currentTime - 2, 29, "ethan"));
-
-        Topologies withNewTopo = addTopologies(topologies,
-            genTopology("rui-topo-1", ruiConf, 1, 0, 4, 0, currentTime - 2, 10, "rui"));
-
-        Config config = mkClusterConfig(DefaultSchedulingPriorityStrategy.class.getName());
-        Cluster cluster = mkTestCluster(topologies, config);
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-
-        scheduler.schedule(topologies, cluster);
-
-        assertTopologiesFullyScheduled(cluster, "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
-
-        cluster = new Cluster(cluster, withNewTopo);
-        scheduler.schedule(withNewTopo, cluster);
-        Map<String, Set<String>> evictedTopos = ((ResourceAwareScheduler) scheduler).getEvictedTopologiesMap();
-
-        assertTopologiesFullyScheduled(cluster, "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
-        assertTopologiesNotBeenEvicted(cluster, collectMapValues(evictedTopos), "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
-        assertTopologiesFullyScheduled(cluster, "rui-topo-1");
+        for (Class strategyClass: strategyClasses) {
+            Map<String, Double> requestedgenericResourcesMap = new HashMap<>();
+            requestedgenericResourcesMap.put("generic.resource.1", 40.0);
+            // Use full memory and cpu of the cluster capacity
+            Config ruiConf = createGrasClusterConfig(strategyClass, 20, 50, 50, null, requestedgenericResourcesMap);
+            Config ethanConf = createGrasClusterConfig(strategyClass, 80, 400, 500, null, Collections.emptyMap());
+            Topologies topologies = new Topologies(
+                    genTopology("ethan-topo-1", ethanConf, 1, 0, 1, 0, currentTime - 2, 10, "ethan"),
+                    genTopology("ethan-topo-2", ethanConf, 1, 0, 1, 0, currentTime - 2, 20, "ethan"),
+                    genTopology("ethan-topo-3", ethanConf, 1, 0, 1, 0, currentTime - 2, 28, "ethan"),
+                    genTopology("ethan-topo-4", ethanConf, 1, 0, 1, 0, currentTime - 2, 29, "ethan"));
+
+            Topologies withNewTopo = addTopologies(topologies,
+                    genTopology("rui-topo-1", ruiConf, 1, 0, 4, 0, currentTime - 2, 10, "rui"));
+
+            Config config = mkClusterConfig(strategyClass, DefaultSchedulingPriorityStrategy.class.getName());
+            Cluster cluster = mkTestCluster(topologies, config);
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+
+            scheduler.schedule(topologies, cluster);
+
+            assertTopologiesFullyScheduled(cluster, strategyClass, "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
+
+            cluster = new Cluster(cluster, withNewTopo);
+            scheduler.schedule(withNewTopo, cluster);
+            Map<String, Set<String>> evictedTopos = ((ResourceAwareScheduler) scheduler).getEvictedTopologiesMap();
+
+            assertTopologiesFullyScheduled(cluster, strategyClass, "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
+            assertTopologiesNotBeenEvicted(cluster, strategyClass, collectMapValues(evictedTopos), "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
+            assertTopologiesFullyScheduled(cluster, strategyClass, "rui-topo-1");
+        }
     }
 
     /*
@@ -124,34 +130,36 @@ public class TestGenericResourceAwareSchedulingPriorityStrategy {
      */
     @Test
     public void testDefaultSchedulingPriorityStrategyEvicting() {
-        Map<String, Double> requestedgenericResourcesMap = new HashMap<>();
-        requestedgenericResourcesMap.put("generic.resource.1", 40.0);
-        Config ruiConf = createGrasClusterConfig(10, 10, 10, null, requestedgenericResourcesMap);
-        Config ethanConf = createGrasClusterConfig(60, 200, 300, null, Collections.emptyMap());
-        Topologies topologies = new Topologies(
-            genTopology("ethan-topo-1", ethanConf, 1, 0, 1, 0, currentTime - 2, 10, "ethan"),
-            genTopology("ethan-topo-2", ethanConf, 1, 0, 1, 0, currentTime - 2, 20, "ethan"),
-            genTopology("ethan-topo-3", ethanConf, 1, 0, 1, 0, currentTime - 2, 28, "ethan"),
-            genTopology("ethan-topo-4", ethanConf, 1, 0, 1, 0, currentTime - 2, 29, "ethan"));
-
-        Topologies withNewTopo = addTopologies(topologies,
-            genTopology("rui-topo-1", ruiConf, 1, 0, 5, 0, currentTime - 2, 10, "rui"));
-
-        Config config = mkClusterConfig(DefaultSchedulingPriorityStrategy.class.getName());
-        Cluster cluster = mkTestCluster(topologies, config);
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-
-        assertTopologiesFullyScheduled(cluster, "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
-
-        cluster = new Cluster(cluster, withNewTopo);
-        scheduler.schedule(withNewTopo, cluster);
-        Map<String, Set<String>> evictedTopos = ((ResourceAwareScheduler) scheduler).getEvictedTopologiesMap();
-
-        assertTopologiesFullyScheduled(cluster, "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
-        assertTopologiesBeenEvicted(cluster, collectMapValues(evictedTopos), "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
-        assertTopologiesNotScheduled(cluster, "rui-topo-1");
+        for (Class strategyClass: strategyClasses) {
+            Map<String, Double> requestedgenericResourcesMap = new HashMap<>();
+            requestedgenericResourcesMap.put("generic.resource.1", 40.0);
+            Config ruiConf = createGrasClusterConfig(strategyClass, 10, 10, 10, null, requestedgenericResourcesMap);
+            Config ethanConf = createGrasClusterConfig(strategyClass, 60, 200, 300, null, Collections.emptyMap());
+            Topologies topologies = new Topologies(
+                    genTopology("ethan-topo-1", ethanConf, 1, 0, 1, 0, currentTime - 2, 10, "ethan"),
+                    genTopology("ethan-topo-2", ethanConf, 1, 0, 1, 0, currentTime - 2, 20, "ethan"),
+                    genTopology("ethan-topo-3", ethanConf, 1, 0, 1, 0, currentTime - 2, 28, "ethan"),
+                    genTopology("ethan-topo-4", ethanConf, 1, 0, 1, 0, currentTime - 2, 29, "ethan"));
+
+            Topologies withNewTopo = addTopologies(topologies,
+                    genTopology("rui-topo-1", ruiConf, 1, 0, 5, 0, currentTime - 2, 10, "rui"));
+
+            Config config = mkClusterConfig(strategyClass, DefaultSchedulingPriorityStrategy.class.getName());
+            Cluster cluster = mkTestCluster(topologies, config);
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+
+            assertTopologiesFullyScheduled(cluster, strategyClass, "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
+
+            cluster = new Cluster(cluster, withNewTopo);
+            scheduler.schedule(withNewTopo, cluster);
+            Map<String, Set<String>> evictedTopos = ((ResourceAwareScheduler) scheduler).getEvictedTopologiesMap();
+
+            assertTopologiesFullyScheduled(cluster, strategyClass, "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
+            assertTopologiesBeenEvicted(cluster, strategyClass, collectMapValues(evictedTopos), "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
+            assertTopologiesNotScheduled(cluster, strategyClass, "rui-topo-1");
+        }
     }
 
     /*
@@ -163,39 +171,41 @@ public class TestGenericResourceAwareSchedulingPriorityStrategy {
      */
     @Test
     public void testGenericSchedulingPriorityStrategyEvicting() {
-        Map<String, Double> requestedgenericResourcesMap = new HashMap<>();
-        requestedgenericResourcesMap.put("generic.resource.1", 40.0);
-        Config ruiConf = createGrasClusterConfig(10, 10, 10, null, requestedgenericResourcesMap);
-        Config ethanConf = createGrasClusterConfig(60, 200, 300, null, Collections.emptyMap());
-        Topologies topologies = new Topologies(
-            genTopology("ethan-topo-1", ethanConf, 1, 0, 1, 0, currentTime - 2, 10, "ethan"),
-            genTopology("ethan-topo-2", ethanConf, 1, 0, 1, 0, currentTime - 2, 20, "ethan"),
-            genTopology("ethan-topo-3", ethanConf, 1, 0, 1, 0, currentTime - 2, 28, "ethan"),
-            genTopology("ethan-topo-4", ethanConf, 1, 0, 1, 0, currentTime - 2, 29, "ethan"));
-
-        Topologies withNewTopo = addTopologies(topologies,
-            genTopology("rui-topo-1", ruiConf, 1, 0, 5, 0, currentTime - 2, 10, "rui"));
-
-
-        Config config = mkClusterConfig(GenericResourceAwareSchedulingPriorityStrategy.class.getName());
-        Cluster cluster = mkTestCluster(topologies, config);
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-
-        assertTopologiesFullyScheduled(cluster, "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
-
-        cluster = new Cluster(cluster, withNewTopo);
-        scheduler.schedule(withNewTopo, cluster);
-        Map<String, Set<String>> evictedTopos = ((ResourceAwareScheduler) scheduler).getEvictedTopologiesMap();
-
-        assertTopologiesFullyScheduled(cluster, "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
-        assertTopologiesNotBeenEvicted(cluster, collectMapValues(evictedTopos),"ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
-        assertTopologiesNotScheduled(cluster, "rui-topo-1");
+        for (Class strategyClass: strategyClasses) {
+            Map<String, Double> requestedgenericResourcesMap = new HashMap<>();
+            requestedgenericResourcesMap.put("generic.resource.1", 40.0);
+            Config ruiConf = createGrasClusterConfig(strategyClass, 10, 10, 10, null, requestedgenericResourcesMap);
+            Config ethanConf = createGrasClusterConfig(strategyClass, 60, 200, 300, null, Collections.emptyMap());
+            Topologies topologies = new Topologies(
+                    genTopology("ethan-topo-1", ethanConf, 1, 0, 1, 0, currentTime - 2, 10, "ethan"),
+                    genTopology("ethan-topo-2", ethanConf, 1, 0, 1, 0, currentTime - 2, 20, "ethan"),
+                    genTopology("ethan-topo-3", ethanConf, 1, 0, 1, 0, currentTime - 2, 28, "ethan"),
+                    genTopology("ethan-topo-4", ethanConf, 1, 0, 1, 0, currentTime - 2, 29, "ethan"));
+
+            Topologies withNewTopo = addTopologies(topologies,
+                    genTopology("rui-topo-1", ruiConf, 1, 0, 5, 0, currentTime - 2, 10, "rui"));
+
+
+            Config config = mkClusterConfig(strategyClass, GenericResourceAwareSchedulingPriorityStrategy.class.getName());
+            Cluster cluster = mkTestCluster(topologies, config);
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+
+            assertTopologiesFullyScheduled(cluster, strategyClass, "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
+
+            cluster = new Cluster(cluster, withNewTopo);
+            scheduler.schedule(withNewTopo, cluster);
+            Map<String, Set<String>> evictedTopos = ((ResourceAwareScheduler) scheduler).getEvictedTopologiesMap();
+
+            assertTopologiesFullyScheduled(cluster, strategyClass, "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
+            assertTopologiesNotBeenEvicted(cluster, strategyClass, collectMapValues(evictedTopos), "ethan-topo-1", "ethan-topo-2", "ethan-topo-3", "ethan-topo-4");
+            assertTopologiesNotScheduled(cluster, strategyClass, "rui-topo-1");
+        }
     }
 
 
-    private Config mkClusterConfig(String SchedulingPriorityStrategy) {
+    private Config mkClusterConfig(Class strategyClass, String SchedulingPriorityStrategy) {
         Map<String, Map<String, Number>> resourceUserPool = userResourcePool(
             userRes("rui", 200, 2000),
             userRes("ethan", 200, 2000));
@@ -203,7 +213,7 @@ public class TestGenericResourceAwareSchedulingPriorityStrategy {
         Map<String, Double> genericResourcesOfferedMap = new HashMap<>();
         genericResourcesOfferedMap.put("generic.resource.1", 50.0);
 
-        Config config = createGrasClusterConfig(100, 500, 500, resourceUserPool, genericResourcesOfferedMap);
+        Config config = createGrasClusterConfig(strategyClass, 100, 500, 500, resourceUserPool, genericResourcesOfferedMap);
         config.put(DaemonConfig.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, SchedulingPriorityStrategy);
         config.put(DaemonConfig.RESOURCE_AWARE_SCHEDULER_MAX_TOPOLOGY_SCHEDULING_ATTEMPTS, 2);    // allow 1 round of evictions
 
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestBackwardCompatibility.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestBackwardCompatibility.java
deleted file mode 100644
index f6ed48dc9..000000000
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestBackwardCompatibility.java
+++ /dev/null
@@ -1,435 +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.storm.scheduler.resource.strategies.scheduling;
-
-import org.apache.storm.TestRebalance;
-import org.apache.storm.daemon.nimbus.NimbusTest;
-import org.apache.storm.generated.InvalidTopologyException;
-import org.apache.storm.scheduler.blacklist.TestBlacklistScheduler;
-import org.apache.storm.scheduler.resource.TestResourceAwareScheduler;
-import org.apache.storm.scheduler.resource.TestUser;
-import org.apache.storm.scheduler.resource.strategies.eviction.TestDefaultEvictionStrategy;
-import org.apache.storm.scheduler.resource.strategies.priority.TestFIFOSchedulingPriorityStrategy;
-import org.apache.storm.scheduler.resource.strategies.priority.TestGenericResourceAwareSchedulingPriorityStrategy;
-import org.apache.storm.testing.PerformanceTest;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.EnumSource;
-
-/**
- * Test for backward compatibility.
- *
- * <p>
- * {@link GenericResourceAwareStrategyOld} class behavior is supposed to be compatible
- * with the prior version of {@link GenericResourceAwareStrategy} and
- * {@link DefaultResourceAwareStrategyOld} class behavior is supposed to be compatible
- * with the prior version of {@link DefaultResourceAwareStrategy}.
- * </p>
- *
- * The tests in this class wrap tests in other classes while replacing Strategy classes.
- * The wrapped classes have protected methods that return strategy classes. These methods
- * are overridden to return backward compatible class.
- */
-public class TestBackwardCompatibility {
-
-    TestGenericResourceAwareStrategy testGenericResourceAwareStrategy;
-    TestResourceAwareScheduler testResourceAwareScheduler;
-    TestBlacklistScheduler testBlacklistScheduler;
-    NimbusTest nimbusTest;
-    TestRebalance testRebalance;
-    TestGenericResourceAwareSchedulingPriorityStrategy testGenericResourceAwareSchedulingPriorityStrategy;
-
-    TestDefaultResourceAwareStrategy testDefaultResourceAwareStrategy;
-    TestFIFOSchedulingPriorityStrategy testFIFOSchedulingPriorityStrategy;
-    TestDefaultEvictionStrategy testDefaultEvictionStrategy;
-    TestUser testUser;
-
-    public TestBackwardCompatibility() {
-        // Create instances of wrapped test classes and override strategy class methods
-        testGenericResourceAwareStrategy = new TestGenericResourceAwareStrategy() {
-            @Override
-            protected Class getGenericResourceAwareStrategyClass() {
-                return GenericResourceAwareStrategyOld.class;
-            }
-        };
-        testResourceAwareScheduler = new TestResourceAwareScheduler() {
-            @Override
-            protected Class getDefaultResourceAwareStrategyClass() {
-                return DefaultResourceAwareStrategyOld.class;
-            }
-
-            @Override
-            protected Class getGenericResourceAwareStrategyClass() {
-                return GenericResourceAwareStrategyOld.class;
-            }
-        };
-        testBlacklistScheduler = new TestBlacklistScheduler() {
-            @Override
-            protected Class getDefaultResourceAwareStrategyClass() {
-                return DefaultResourceAwareStrategyOld.class;
-            }
-        };
-        nimbusTest = new NimbusTest() {
-            @Override
-            protected Class getDefaultResourceAwareStrategyClass() {
-                return DefaultResourceAwareStrategyOld.class;
-            }
-        };
-        testRebalance = new TestRebalance() {
-            @Override
-            protected Class getDefaultResourceAwareStrategyClass() {
-                return DefaultResourceAwareStrategyOld.class;
-            }
-        };
-        testGenericResourceAwareSchedulingPriorityStrategy = new TestGenericResourceAwareSchedulingPriorityStrategy() {
-            @Override
-            protected Class getGenericResourceAwareStrategyClass() {
-                return GenericResourceAwareStrategyOld.class;
-            }
-        };
-        testDefaultResourceAwareStrategy = new TestDefaultResourceAwareStrategy() {
-            @Override
-            protected Class getDefaultResourceAwareStrategyClass() {
-                return DefaultResourceAwareStrategyOld.class;
-            }
-        };
-        testFIFOSchedulingPriorityStrategy = new TestFIFOSchedulingPriorityStrategy() {
-            @Override
-            protected Class getDefaultResourceAwareStrategyClass() {
-                return DefaultResourceAwareStrategyOld.class;
-            }
-        };
-        testDefaultEvictionStrategy = new TestDefaultEvictionStrategy() {
-            @Override
-            protected Class getDefaultResourceAwareStrategyClass() {
-                return DefaultResourceAwareStrategyOld.class;
-            }
-        };
-        testUser = new TestUser() {
-            @Override
-            protected Class getDefaultResourceAwareStrategyClass() {
-                return DefaultResourceAwareStrategyOld.class;
-            }
-        };
-
-    }
-
-    /**********************************************************************************
-     *  Tests for  testGenericResourceAwareStrategy
-     ***********************************************************************************/
-
-    @Test
-    public void testGenericResourceAwareStrategySharedMemory() {
-        testGenericResourceAwareStrategy.testGenericResourceAwareStrategySharedMemory();
-    }
-
-    @Test
-    public void testGenericResourceAwareStrategy()
-        throws InvalidTopologyException {
-        testGenericResourceAwareStrategy.testGenericResourceAwareStrategyWithoutSettingAckerExecutors(0);
-    }
-
-    @Test
-    public void testGenericResourceAwareStrategyInFavorOfShuffle()
-        throws InvalidTopologyException {
-        testGenericResourceAwareStrategy.testGenericResourceAwareStrategyInFavorOfShuffle();
-    }
-
-    @Test
-    public void testGrasRequiringEviction() {
-        testGenericResourceAwareStrategy.testGrasRequiringEviction();
-    }
-
-    @Test
-    public void testAntiAffinityWithMultipleTopologies() {
-        testGenericResourceAwareStrategy.testAntiAffinityWithMultipleTopologies();
-    }
-
-    /**********************************************************************************
-     *  Tests for  testResourceAwareScheduler
-     ***********************************************************************************/
-
-    @PerformanceTest
-    @Test
-    public void testLargeTopologiesOnLargeClusters() {
-        testResourceAwareScheduler.testLargeTopologiesOnLargeClusters();
-    }
-
-    @PerformanceTest
-    @Test
-    public void testLargeTopologiesOnLargeClustersGras() {
-        testResourceAwareScheduler.testLargeTopologiesOnLargeClustersGras();
-    }
-
-    @Test
-    public void testHeterogeneousClusterwithGras() {
-        testResourceAwareScheduler.testHeterogeneousClusterwithGras();
-    }
-
-    @Test
-    public void testRASNodeSlotAssign() {
-        testResourceAwareScheduler.testRASNodeSlotAssign();
-    }
-
-    @Test
-    public void sanityTestOfScheduling() {
-        testResourceAwareScheduler.sanityTestOfScheduling();
-    }
-
-    @Test
-    public void testTopologyWithMultipleSpouts() {
-        testResourceAwareScheduler.testTopologyWithMultipleSpouts();
-    }
-
-    @Test
-    public void testTopologySetCpuAndMemLoad() {
-        testResourceAwareScheduler.testTopologySetCpuAndMemLoad();
-    }
-
-    @Test
-    public void testResourceLimitation() {
-        testResourceAwareScheduler.testResourceLimitation();
-    }
-
-    @Test
-    public void testScheduleResilience() {
-        testResourceAwareScheduler.testScheduleResilience();
-    }
-
-    @Test
-    public void testHeterogeneousClusterwithDefaultRas() {
-        testResourceAwareScheduler.testHeterogeneousClusterwithDefaultRas();
-    }
-
-    @Test
-    public void testTopologyWorkerMaxHeapSize() {
-        testResourceAwareScheduler.testTopologyWorkerMaxHeapSize();
-    }
-
-    @Test
-    public void testReadInResourceAwareSchedulerUserPools() {
-        testResourceAwareScheduler.testReadInResourceAwareSchedulerUserPools();
-    }
-
-    @Test
-    public void testSubmitUsersWithNoGuarantees() {
-        testResourceAwareScheduler.testSubmitUsersWithNoGuarantees();
-    }
-
-    @Test
-    public void testMultipleUsers() {
-        testResourceAwareScheduler.testMultipleUsers();
-    }
-
-    @Test
-    public void testHandlingClusterSubscription() {
-        testResourceAwareScheduler.testHandlingClusterSubscription();
-    }
-
-    @Test
-    public void testFaultTolerance() {
-        testResourceAwareScheduler.testFaultTolerance();
-    }
-
-    @Test
-    public void testNodeFreeSlot() {
-        testResourceAwareScheduler.testNodeFreeSlot();
-    }
-
-    @Test
-    public void testSchedulingAfterFailedScheduling() {
-        testResourceAwareScheduler.testSchedulingAfterFailedScheduling();
-    }
-
-    @Test
-    public void minCpuWorkerJustFits() {
-        testResourceAwareScheduler.minCpuWorkerJustFits();
-    }
-
-    @Test
-    public void minCpuPreventsThirdTopo() {
-        testResourceAwareScheduler.minCpuPreventsThirdTopo();
-    }
-
-    @Test
-    public void testMinCpuMaxMultipleSupervisors() {
-        testResourceAwareScheduler.testMinCpuMaxMultipleSupervisors();
-    }
-
-    @Test
-    public void minCpuWorkerSplitFails() {
-        testResourceAwareScheduler.minCpuWorkerSplitFails();
-    }
-
-    @Test
-    public void TestLargeFragmentedClusterScheduling() {
-        testResourceAwareScheduler.TestLargeFragmentedClusterScheduling();
-    }
-
-    @Test
-    public void testMultipleSpoutsAndCyclicTopologies() {
-        testResourceAwareScheduler.testMultipleSpoutsAndCyclicTopologies();
-    }
-
-    @Test
-    public void testSchedulerStrategyWhitelist() {
-        testResourceAwareScheduler.testSchedulerStrategyWhitelist();
-    }
-
-    @Test
-    public void testSchedulerStrategyWhitelistException() {
-        testResourceAwareScheduler.testSchedulerStrategyWhitelistException();
-    }
-
-    @Test
-    public void testSchedulerStrategyEmptyWhitelist() {
-        testResourceAwareScheduler.testSchedulerStrategyEmptyWhitelist();
-    }
-
-    @Test
-    public void testStrategyTakingTooLong() {
-        testResourceAwareScheduler.testStrategyTakingTooLong();
-    }
-
-    /**********************************************************************************
-     *  Tests for  TestBlackListScheduler
-     ***********************************************************************************/
-    @Test
-    public void TestGreylist() {
-        testBlacklistScheduler.TestGreylist();
-    }
-
-    /**********************************************************************************
-     *  Tests for  NimbusTest
-     ***********************************************************************************/
-    @Test
-    public void testMemoryLoadLargerThanMaxHeapSize() throws Exception {
-        nimbusTest.testMemoryLoadLargerThanMaxHeapSize();
-    }
-
-    /**********************************************************************************
-     *  Tests for  TestRebalance
-     ***********************************************************************************/
-    @Test
-    public void testRebalanceTopologyResourcesAndConfigs() throws Exception {
-        testRebalance.testRebalanceTopologyResourcesAndConfigs();
-    }
-
-    /**********************************************************************************
-     *  Tests for  testGenericResourceAwareSchedulingPriorityStrategy
-     ***********************************************************************************/
-    @Test
-    public void testDefaultSchedulingPriorityStrategyNotEvicting() {
-        testGenericResourceAwareSchedulingPriorityStrategy.testDefaultSchedulingPriorityStrategyNotEvicting();
-    }
-
-    @Test
-    public void testDefaultSchedulingPriorityStrategyEvicting() {
-        testGenericResourceAwareSchedulingPriorityStrategy.testDefaultSchedulingPriorityStrategyEvicting();
-    }
-
-    @Test
-    public void testGenericSchedulingPriorityStrategyEvicting() {
-        testGenericResourceAwareSchedulingPriorityStrategy.testGenericSchedulingPriorityStrategyEvicting();
-    }
-
-    /**********************************************************************************
-     *  Tests for  testDefaultResourceAwareStrategy
-     ***********************************************************************************/
-
-    @Test
-    public void testSchedulingNegativeResources() {
-        testDefaultResourceAwareStrategy.testSchedulingNegativeResources();
-    }
-
-    @ParameterizedTest
-    @EnumSource(TestDefaultResourceAwareStrategy.WorkerRestrictionType.class)
-    public void testDefaultResourceAwareStrategySharedMemory(TestDefaultResourceAwareStrategy.WorkerRestrictionType schedulingLimitation) {
-        testDefaultResourceAwareStrategy.testDefaultResourceAwareStrategySharedMemory(schedulingLimitation);
-    }
-
-    @Test
-    public void testDefaultResourceAwareStrategy()
-        throws InvalidTopologyException {
-        testDefaultResourceAwareStrategy.testDefaultResourceAwareStrategyWithoutSettingAckerExecutors(0);
-    }
-
-    @Test
-    public void testDefaultResourceAwareStrategyInFavorOfShuffle()
-        throws InvalidTopologyException {
-        testDefaultResourceAwareStrategy.testDefaultResourceAwareStrategyInFavorOfShuffle();
-    }
-
-    @Test
-    public void testMultipleRacks() {
-        testDefaultResourceAwareStrategy.testMultipleRacks();
-    }
-
-    @Test
-    public void testMultipleRacksWithFavoritism() {
-        testDefaultResourceAwareStrategy.testMultipleRacksWithFavoritism();
-    }
-
-    /**********************************************************************************
-     *  Tests for  TestFIFOSchedulingPriorityStrategy
-     ***********************************************************************************/
-
-    @Test
-    public void testFIFOEvictionStrategy() {
-        testFIFOSchedulingPriorityStrategy.testFIFOEvictionStrategy();
-    }
-
-    /**********************************************************************************
-     *  Tests for  TestDefaultEvictionStrategy
-     ***********************************************************************************/
-
-    @Test
-    public void testEviction() {
-        testDefaultEvictionStrategy.testEviction();
-    }
-
-    @Test
-    public void testEvictMultipleTopologies() {
-        testDefaultEvictionStrategy.testEvictMultipleTopologies();
-    }
-
-    @Test
-    public void testEvictMultipleTopologiesFromMultipleUsersInCorrectOrder() {
-        testDefaultEvictionStrategy.testEvictMultipleTopologiesFromMultipleUsersInCorrectOrder();
-    }
-
-    @Test
-    public void testEvictTopologyFromItself() {
-        testDefaultEvictionStrategy.testEvictTopologyFromItself();
-    }
-
-    @Test
-    public void testOverGuaranteeEviction() {
-        testDefaultEvictionStrategy.testOverGuaranteeEviction();
-    }
-
-    /**********************************************************************************
-     *  Tests for  TestUser
-     ***********************************************************************************/
-
-    @Test
-    public void testResourcePoolUtilization() {
-        testUser.testResourcePoolUtilization();
-    }
-}
\ No newline at end of file
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestDefaultResourceAwareStrategy.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestDefaultResourceAwareStrategy.java
index 1f5d9bdca..1b8dae8f1 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestDefaultResourceAwareStrategy.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestDefaultResourceAwareStrategy.java
@@ -87,6 +87,10 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
 
 @ExtendWith({NormalizedResourcesExtension.class})
 public class TestDefaultResourceAwareStrategy {
+    private static final Class[] strategyClasses = {
+            DefaultResourceAwareStrategy.class,
+            DefaultResourceAwareStrategyOld.class,
+    };
     private static final int CURRENT_TIME = 1450418597;
     private static IScheduler scheduler = null;
     private enum SharedMemoryType {
@@ -100,14 +104,10 @@ public class TestDefaultResourceAwareStrategy {
         WORKER_RESTRICTION_NONE
     }
 
-    protected Class getDefaultResourceAwareStrategyClass() {
-        return DefaultResourceAwareStrategy.class;
-    }
-
-    private Config createClusterConfig(double compPcore, double compOnHeap, double compOffHeap,
-                                             Map<String, Map<String, Number>> pools) {
+    private Config createClusterConfig(Class strategyClass, double compPcore, double compOnHeap, double compOffHeap,
+                                       Map<String, Map<String, Number>> pools) {
         Config config = TestUtilsForResourceAwareScheduler.createClusterConfig(compPcore, compOnHeap, compOffHeap, pools);
-        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, getDefaultResourceAwareStrategyClass().getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, strategyClass.getName());
         return config;
     }
 
@@ -145,86 +145,88 @@ public class TestDefaultResourceAwareStrategy {
     @ParameterizedTest
     @EnumSource(SharedMemoryType.class)
     public void testMultipleSharedMemoryWithOneExecutorPerWorker(SharedMemoryType memoryType) {
-        int spoutParallelism = 4;
-        double cpuPercent = 10;
-        double memoryOnHeap = 10;
-        double memoryOffHeap = 10;
-        double sharedOnHeapWithinWorker = 450;
-        double sharedOffHeapWithinNode = 600;
-        double sharedOffHeapWithinWorker = 400;
-
-        TopologyBuilder builder = new TopologyBuilder();
-        switch (memoryType) {
-            case SHARED_OFF_HEAP_NODE:
-                builder.setSpout("spout", new TestSpout(), spoutParallelism)
-                        .addSharedMemory(new SharedOffHeapWithinNode(sharedOffHeapWithinNode, "spout shared off heap within node"));
-                break;
-            case SHARED_OFF_HEAP_WORKER:
-                builder.setSpout("spout", new TestSpout(), spoutParallelism)
-                        .addSharedMemory(new SharedOffHeapWithinWorker(sharedOffHeapWithinWorker, "spout shared off heap within worker"));
-                break;
-            case SHARED_ON_HEAP_WORKER:
-                builder.setSpout("spout", new TestSpout(), spoutParallelism)
-                        .addSharedMemory(new SharedOnHeap(sharedOnHeapWithinWorker, "spout shared on heap within worker"));
-                break;
-        }
-        StormTopology stormToplogy = builder.createTopology();
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 500, 1000);
-        Config conf = createClusterConfig(cpuPercent, memoryOnHeap, memoryOffHeap, null);
-
-        conf.put(Config.TOPOLOGY_PRIORITY, 0);
-        conf.put(Config.TOPOLOGY_NAME, "testTopology");
-        conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
-        conf.put(Config.TOPOLOGY_RAS_ONE_EXECUTOR_PER_WORKER, true);
-        TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormToplogy, 0,
-                genExecsAndComps(stormToplogy), CURRENT_TIME, "user");
-
-        Topologies topologies = new Topologies(topo);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
-
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(conf, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-
-        TopologyResources topologyResources = cluster.getTopologyResourcesMap().get(topo.getId());
-        SchedulerAssignment assignment = cluster.getAssignmentById(topo.getId());
-        long numNodes = assignment.getSlotToExecutors().keySet().stream().map(WorkerSlot::getNodeId).distinct().count();
-
-        switch (memoryType) {
-            case SHARED_OFF_HEAP_NODE:
-                // 4 workers on single node. OffHeapNode memory is shared
-                assertThat(topologyResources.getAssignedMemOnHeap(), closeTo(spoutParallelism * memoryOnHeap, 0.01));
-                assertThat(topologyResources.getAssignedMemOffHeap(), closeTo(spoutParallelism * memoryOffHeap + sharedOffHeapWithinNode, 0.01));
-                assertThat(topologyResources.getAssignedSharedMemOnHeap(), closeTo(0, 0.01));
-                assertThat(topologyResources.getAssignedSharedMemOffHeap(), closeTo(sharedOffHeapWithinNode, 0.01));
-                assertThat(topologyResources.getAssignedNonSharedMemOnHeap(), closeTo(spoutParallelism * memoryOnHeap, 0.01));
-                assertThat(topologyResources.getAssignedNonSharedMemOffHeap(), closeTo(spoutParallelism * memoryOffHeap, 0.01));
-                assertThat(numNodes, is(1L));
-                assertThat(cluster.getAssignedNumWorkers(topo), is(spoutParallelism));
-                break;
-            case SHARED_OFF_HEAP_WORKER:
-                // 4 workers on 2 nodes. OffHeapWorker memory not shared -- consumed 4x, once for each worker
-                assertThat(topologyResources.getAssignedMemOnHeap(), closeTo(spoutParallelism * memoryOnHeap, 0.01));
-                assertThat(topologyResources.getAssignedMemOffHeap(), closeTo(spoutParallelism * (memoryOffHeap + sharedOffHeapWithinWorker), 0.01));
-                assertThat(topologyResources.getAssignedSharedMemOnHeap(), closeTo(0, 0.01));
-                assertThat(topologyResources.getAssignedSharedMemOffHeap(), closeTo(spoutParallelism * sharedOffHeapWithinWorker, 0.01));
-                assertThat(topologyResources.getAssignedNonSharedMemOnHeap(), closeTo(spoutParallelism * memoryOnHeap, 0.01));
-                assertThat(topologyResources.getAssignedNonSharedMemOffHeap(), closeTo(spoutParallelism * memoryOffHeap, 0.01));
-                assertThat(numNodes, is(2L));
-                assertThat(cluster.getAssignedNumWorkers(topo), is(spoutParallelism));
-                break;
-            case SHARED_ON_HEAP_WORKER:
-                // 4 workers on 2 nodes. onHeap memory not shared -- consumed 4x, once for each worker
-                assertThat(topologyResources.getAssignedMemOnHeap(), closeTo(spoutParallelism * (memoryOnHeap + sharedOnHeapWithinWorker), 0.01));
-                assertThat(topologyResources.getAssignedMemOffHeap(), closeTo(spoutParallelism * memoryOffHeap, 0.01));
-                assertThat(topologyResources.getAssignedSharedMemOnHeap(), closeTo(spoutParallelism * sharedOnHeapWithinWorker, 0.01));
-                assertThat(topologyResources.getAssignedSharedMemOffHeap(), closeTo(0, 0.01));
-                assertThat(topologyResources.getAssignedNonSharedMemOnHeap(), closeTo(spoutParallelism * memoryOnHeap, 0.01));
-                assertThat(topologyResources.getAssignedNonSharedMemOffHeap(), closeTo(spoutParallelism * memoryOffHeap, 0.01));
-                assertThat(numNodes, is(2L));
-                assertThat(cluster.getAssignedNumWorkers(topo), is(spoutParallelism));
-                break;
+        for (Class strategyClass: strategyClasses) {
+            int spoutParallelism = 4;
+            double cpuPercent = 10;
+            double memoryOnHeap = 10;
+            double memoryOffHeap = 10;
+            double sharedOnHeapWithinWorker = 450;
+            double sharedOffHeapWithinNode = 600;
+            double sharedOffHeapWithinWorker = 400;
+
+            TopologyBuilder builder = new TopologyBuilder();
+            switch (memoryType) {
+                case SHARED_OFF_HEAP_NODE:
+                    builder.setSpout("spout", new TestSpout(), spoutParallelism)
+                            .addSharedMemory(new SharedOffHeapWithinNode(sharedOffHeapWithinNode, "spout shared off heap within node"));
+                    break;
+                case SHARED_OFF_HEAP_WORKER:
+                    builder.setSpout("spout", new TestSpout(), spoutParallelism)
+                            .addSharedMemory(new SharedOffHeapWithinWorker(sharedOffHeapWithinWorker, "spout shared off heap within worker"));
+                    break;
+                case SHARED_ON_HEAP_WORKER:
+                    builder.setSpout("spout", new TestSpout(), spoutParallelism)
+                            .addSharedMemory(new SharedOnHeap(sharedOnHeapWithinWorker, "spout shared on heap within worker"));
+                    break;
+            }
+            StormTopology stormToplogy = builder.createTopology();
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 500, 1000);
+            Config conf = createClusterConfig(strategyClass, cpuPercent, memoryOnHeap, memoryOffHeap, null);
+
+            conf.put(Config.TOPOLOGY_PRIORITY, 0);
+            conf.put(Config.TOPOLOGY_NAME, "testTopology");
+            conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
+            conf.put(Config.TOPOLOGY_RAS_ONE_EXECUTOR_PER_WORKER, true);
+            TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormToplogy, 0,
+                    genExecsAndComps(stormToplogy), CURRENT_TIME, "user");
+
+            Topologies topologies = new Topologies(topo);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
+
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(conf, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+
+            TopologyResources topologyResources = cluster.getTopologyResourcesMap().get(topo.getId());
+            SchedulerAssignment assignment = cluster.getAssignmentById(topo.getId());
+            long numNodes = assignment.getSlotToExecutors().keySet().stream().map(WorkerSlot::getNodeId).distinct().count();
+
+            switch (memoryType) {
+                case SHARED_OFF_HEAP_NODE:
+                    // 4 workers on single node. OffHeapNode memory is shared
+                    assertThat(topologyResources.getAssignedMemOnHeap(), closeTo(spoutParallelism * memoryOnHeap, 0.01));
+                    assertThat(topologyResources.getAssignedMemOffHeap(), closeTo(spoutParallelism * memoryOffHeap + sharedOffHeapWithinNode, 0.01));
+                    assertThat(topologyResources.getAssignedSharedMemOnHeap(), closeTo(0, 0.01));
+                    assertThat(topologyResources.getAssignedSharedMemOffHeap(), closeTo(sharedOffHeapWithinNode, 0.01));
+                    assertThat(topologyResources.getAssignedNonSharedMemOnHeap(), closeTo(spoutParallelism * memoryOnHeap, 0.01));
+                    assertThat(topologyResources.getAssignedNonSharedMemOffHeap(), closeTo(spoutParallelism * memoryOffHeap, 0.01));
+                    assertThat(numNodes, is(1L));
+                    assertThat(cluster.getAssignedNumWorkers(topo), is(spoutParallelism));
+                    break;
+                case SHARED_OFF_HEAP_WORKER:
+                    // 4 workers on 2 nodes. OffHeapWorker memory not shared -- consumed 4x, once for each worker
+                    assertThat(topologyResources.getAssignedMemOnHeap(), closeTo(spoutParallelism * memoryOnHeap, 0.01));
+                    assertThat(topologyResources.getAssignedMemOffHeap(), closeTo(spoutParallelism * (memoryOffHeap + sharedOffHeapWithinWorker), 0.01));
+                    assertThat(topologyResources.getAssignedSharedMemOnHeap(), closeTo(0, 0.01));
+                    assertThat(topologyResources.getAssignedSharedMemOffHeap(), closeTo(spoutParallelism * sharedOffHeapWithinWorker, 0.01));
+                    assertThat(topologyResources.getAssignedNonSharedMemOnHeap(), closeTo(spoutParallelism * memoryOnHeap, 0.01));
+                    assertThat(topologyResources.getAssignedNonSharedMemOffHeap(), closeTo(spoutParallelism * memoryOffHeap, 0.01));
+                    assertThat(numNodes, is(2L));
+                    assertThat(cluster.getAssignedNumWorkers(topo), is(spoutParallelism));
+                    break;
+                case SHARED_ON_HEAP_WORKER:
+                    // 4 workers on 2 nodes. onHeap memory not shared -- consumed 4x, once for each worker
+                    assertThat(topologyResources.getAssignedMemOnHeap(), closeTo(spoutParallelism * (memoryOnHeap + sharedOnHeapWithinWorker), 0.01));
+                    assertThat(topologyResources.getAssignedMemOffHeap(), closeTo(spoutParallelism * memoryOffHeap, 0.01));
+                    assertThat(topologyResources.getAssignedSharedMemOnHeap(), closeTo(spoutParallelism * sharedOnHeapWithinWorker, 0.01));
+                    assertThat(topologyResources.getAssignedSharedMemOffHeap(), closeTo(0, 0.01));
+                    assertThat(topologyResources.getAssignedNonSharedMemOnHeap(), closeTo(spoutParallelism * memoryOnHeap, 0.01));
+                    assertThat(topologyResources.getAssignedNonSharedMemOffHeap(), closeTo(spoutParallelism * memoryOffHeap, 0.01));
+                    assertThat(numNodes, is(2L));
+                    assertThat(cluster.getAssignedNumWorkers(topo), is(spoutParallelism));
+                    break;
+            }
         }
     }
 
@@ -233,68 +235,70 @@ public class TestDefaultResourceAwareStrategy {
      */
     @Test
     public void testSchedulingNegativeResources() {
-        int spoutParallelism = 2;
-        int boltParallelism = 2;
-        double cpuPercent = 10;
-        double memoryOnHeap = 10;
-        double memoryOffHeap = 10;
-        double sharedOnHeapWithinWorker = 400;
-        double sharedOffHeapWithinNode = 700;
-        double sharedOffHeapWithinWorker = 500;
-
-        Config conf = createClusterConfig(cpuPercent, memoryOnHeap, memoryOffHeap, null);
-        TopologyDetails[] topo = new TopologyDetails[2];
-
-        // 1st topology
-        TopologyBuilder builder = new TopologyBuilder();
-        builder.setSpout("spout", new TestSpout(),
-                spoutParallelism);
-        builder.setBolt("bolt-1", new TestBolt(),
-                boltParallelism).addSharedMemory(new SharedOffHeapWithinWorker(sharedOffHeapWithinWorker, "bolt-1 shared off heap within worker")).shuffleGrouping("spout");
-        builder.setBolt("bolt-2", new TestBolt(),
-                boltParallelism).addSharedMemory(new SharedOffHeapWithinNode(sharedOffHeapWithinNode, "bolt-2 shared off heap within node")).shuffleGrouping("bolt-1");
-        builder.setBolt("bolt-3", new TestBolt(),
-                boltParallelism).addSharedMemory(new SharedOnHeap(sharedOnHeapWithinWorker, "bolt-3 shared on heap within worker")).shuffleGrouping("bolt-2");
-        StormTopology stormTopology = builder.createTopology();
-
-        conf.put(Config.TOPOLOGY_PRIORITY, 1);
-        conf.put(Config.TOPOLOGY_NAME, "testTopology-0");
-        conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
-        topo[0] = new TopologyDetails("testTopology-id-0", conf, stormTopology, 0,
-                genExecsAndComps(stormTopology), CURRENT_TIME, "user");
-
-        // 2nd topology
-        builder = new TopologyBuilder();
-        builder.setSpout("spout", new TestSpout(),
-                spoutParallelism).addSharedMemory(new SharedOffHeapWithinNode(sharedOffHeapWithinNode, "spout shared off heap within node"));
-        stormTopology = builder.createTopology();
-
-        conf.put(Config.TOPOLOGY_PRIORITY, 0);
-        conf.put(Config.TOPOLOGY_NAME, "testTopology-1");
-        topo[1] = new TopologyDetails("testTopology-id-1", conf, stormTopology, 0,
-                genExecsAndComps(stormTopology), CURRENT_TIME, "user");
-
-        Map<String, SupervisorDetails> supMap = genSupervisors(1, 4, 500, 2000);
-        Topologies topologies = new Topologies(topo[0]);
-        Cluster cluster = new Cluster(new INimbusTest(), new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
-
-        // schedule 1st topology
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(conf, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-        assertTopologiesFullyScheduled(cluster, topo[0].getName());
-
-        // attempt scheduling both topologies.
-        // this triggered negative resource event as the second topology incorrectly scheduled with the first in place
-        // first topology should get evicted for higher priority (lower value) second topology to successfully schedule
-        topologies = new Topologies(topo[0], topo[1]);
-        cluster = new Cluster(cluster, topologies);
-        scheduler.schedule(topologies, cluster);
-        assertTopologiesNotScheduled(cluster, topo[0].getName());
-        assertTopologiesFullyScheduled(cluster, topo[1].getName());
-
-        // check negative resource count
-        assertThat(cluster.getResourceMetrics().getNegativeResourceEventsMeter().getCount(), is(0L));
+        for (Class strategyClass: strategyClasses) {
+            int spoutParallelism = 2;
+            int boltParallelism = 2;
+            double cpuPercent = 10;
+            double memoryOnHeap = 10;
+            double memoryOffHeap = 10;
+            double sharedOnHeapWithinWorker = 400;
+            double sharedOffHeapWithinNode = 700;
+            double sharedOffHeapWithinWorker = 500;
+
+            Config conf = createClusterConfig(strategyClass, cpuPercent, memoryOnHeap, memoryOffHeap, null);
+            TopologyDetails[] topo = new TopologyDetails[2];
+
+            // 1st topology
+            TopologyBuilder builder = new TopologyBuilder();
+            builder.setSpout("spout", new TestSpout(),
+                    spoutParallelism);
+            builder.setBolt("bolt-1", new TestBolt(),
+                    boltParallelism).addSharedMemory(new SharedOffHeapWithinWorker(sharedOffHeapWithinWorker, "bolt-1 shared off heap within worker")).shuffleGrouping("spout");
+            builder.setBolt("bolt-2", new TestBolt(),
+                    boltParallelism).addSharedMemory(new SharedOffHeapWithinNode(sharedOffHeapWithinNode, "bolt-2 shared off heap within node")).shuffleGrouping("bolt-1");
+            builder.setBolt("bolt-3", new TestBolt(),
+                    boltParallelism).addSharedMemory(new SharedOnHeap(sharedOnHeapWithinWorker, "bolt-3 shared on heap within worker")).shuffleGrouping("bolt-2");
+            StormTopology stormTopology = builder.createTopology();
+
+            conf.put(Config.TOPOLOGY_PRIORITY, 1);
+            conf.put(Config.TOPOLOGY_NAME, "testTopology-0");
+            conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
+            topo[0] = new TopologyDetails("testTopology-id-0", conf, stormTopology, 0,
+                    genExecsAndComps(stormTopology), CURRENT_TIME, "user");
+
+            // 2nd topology
+            builder = new TopologyBuilder();
+            builder.setSpout("spout", new TestSpout(),
+                    spoutParallelism).addSharedMemory(new SharedOffHeapWithinNode(sharedOffHeapWithinNode, "spout shared off heap within node"));
+            stormTopology = builder.createTopology();
+
+            conf.put(Config.TOPOLOGY_PRIORITY, 0);
+            conf.put(Config.TOPOLOGY_NAME, "testTopology-1");
+            topo[1] = new TopologyDetails("testTopology-id-1", conf, stormTopology, 0,
+                    genExecsAndComps(stormTopology), CURRENT_TIME, "user");
+
+            Map<String, SupervisorDetails> supMap = genSupervisors(1, 4, 500, 2000);
+            Topologies topologies = new Topologies(topo[0]);
+            Cluster cluster = new Cluster(new INimbusTest(), new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
+
+            // schedule 1st topology
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(conf, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+            assertTopologiesFullyScheduled(cluster, strategyClass, topo[0].getName());
+
+            // attempt scheduling both topologies.
+            // this triggered negative resource event as the second topology incorrectly scheduled with the first in place
+            // first topology should get evicted for higher priority (lower value) second topology to successfully schedule
+            topologies = new Topologies(topo[0], topo[1]);
+            cluster = new Cluster(cluster, topologies);
+            scheduler.schedule(topologies, cluster);
+            assertTopologiesNotScheduled(cluster, strategyClass, topo[0].getName());
+            assertTopologiesFullyScheduled(cluster, strategyClass, topo[1].getName());
+
+            // check negative resource count
+            assertThat(cluster.getResourceMetrics().getNegativeResourceEventsMeter().getCount(), is(0L));
+        }
     }
 
     /**
@@ -303,149 +307,151 @@ public class TestDefaultResourceAwareStrategy {
     @ParameterizedTest
     @EnumSource(WorkerRestrictionType.class)
     public void testDefaultResourceAwareStrategySharedMemory(WorkerRestrictionType schedulingLimitation) {
-        int spoutParallelism = 2;
-        int boltParallelism = 2;
-        int numBolts = 3;
-        double cpuPercent = 10;
-        double memoryOnHeap = 10;
-        double memoryOffHeap = 10;
-        double sharedOnHeapWithinWorker = 400;
-        double sharedOffHeapWithinNode = 700;
-        double sharedOffHeapWithinWorker = 600;
-
-        TopologyBuilder builder = new TopologyBuilder();
-        builder.setSpout("spout", new TestSpout(),
-                spoutParallelism);
-        builder.setBolt("bolt-1", new TestBolt(),
-                boltParallelism).addSharedMemory(new SharedOffHeapWithinWorker(sharedOffHeapWithinWorker, "bolt-1 shared off heap within worker")).shuffleGrouping("spout");
-        builder.setBolt("bolt-2", new TestBolt(),
-                boltParallelism).addSharedMemory(new SharedOffHeapWithinNode(sharedOffHeapWithinNode, "bolt-2 shared off heap within node")).shuffleGrouping("bolt-1");
-        builder.setBolt("bolt-3", new TestBolt(),
-                boltParallelism).addSharedMemory(new SharedOnHeap(sharedOnHeapWithinWorker, "bolt-3 shared on heap within worker")).shuffleGrouping("bolt-2");
-
-        StormTopology stormTopology = builder.createTopology();
-
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 500, 2000);
-        Config conf = createClusterConfig(cpuPercent, memoryOnHeap, memoryOffHeap, null);
-
-        conf.put(Config.TOPOLOGY_PRIORITY, 0);
-        conf.put(Config.TOPOLOGY_NAME, "testTopology");
-        conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
-        switch (schedulingLimitation) {
-            case WORKER_RESTRICTION_ONE_EXECUTOR:
-                conf.put(Config.TOPOLOGY_RAS_ONE_EXECUTOR_PER_WORKER, true);
-                break;
-            case WORKER_RESTRICTION_ONE_COMPONENT:
-                conf.put(Config.TOPOLOGY_RAS_ONE_COMPONENT_PER_WORKER, true);
-                break;
-        }
-        TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormTopology, 0,
-                genExecsAndComps(stormTopology), CURRENT_TIME, "user");
-
-        Topologies topologies = new Topologies(topo);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
-
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(conf, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-
-        // [3,3] [7,7], [0,0] [2,2] [6,6] [1,1] [5,5] [4,4] sorted executor ordering
-        // spout  [0,0] [1,1]
-        // bolt-1 [2,2] [3,3]
-        // bolt-2 [6,6] [7,7]
-        // bolt-3 [4,4] [5,5]
-
-        // WorkerRestrictionType.WORKER_RESTRICTION_NONE
-        // expect 1 worker, 1 node
-
-        // WorkerRestrictionType.WORKER_RESTRICTION_ONE_EXECUTOR
-        // expect 8 workers, 2 nodes
-        // node r000s000 workers: bolt-1 bolt-2 spout bolt-1 (no memory sharing)
-        // node r000s001 workers: bolt-2 spout bolt-3 bolt-3 (no memory sharing)
-
-        // WorkerRestrictionType.WORKER_RESTRICTION_ONE_COMPONENT
-        // expect 4 workers, 1 node
-
-        for (Entry<String, SupervisorResources> entry: cluster.getSupervisorsResourcesMap().entrySet()) {
-            String supervisorId = entry.getKey();
-            SupervisorResources resources = entry.getValue();
-            assertTrue(resources.getTotalCpu() >= resources.getUsedCpu(), supervisorId);
-            assertTrue(resources.getTotalMem() >= resources.getUsedMem(), supervisorId);
-        }
+        for (Class strategyClass: strategyClasses) {
+            int spoutParallelism = 2;
+            int boltParallelism = 2;
+            int numBolts = 3;
+            double cpuPercent = 10;
+            double memoryOnHeap = 10;
+            double memoryOffHeap = 10;
+            double sharedOnHeapWithinWorker = 400;
+            double sharedOffHeapWithinNode = 700;
+            double sharedOffHeapWithinWorker = 600;
+
+            TopologyBuilder builder = new TopologyBuilder();
+            builder.setSpout("spout", new TestSpout(),
+                    spoutParallelism);
+            builder.setBolt("bolt-1", new TestBolt(),
+                    boltParallelism).addSharedMemory(new SharedOffHeapWithinWorker(sharedOffHeapWithinWorker, "bolt-1 shared off heap within worker")).shuffleGrouping("spout");
+            builder.setBolt("bolt-2", new TestBolt(),
+                    boltParallelism).addSharedMemory(new SharedOffHeapWithinNode(sharedOffHeapWithinNode, "bolt-2 shared off heap within node")).shuffleGrouping("bolt-1");
+            builder.setBolt("bolt-3", new TestBolt(),
+                    boltParallelism).addSharedMemory(new SharedOnHeap(sharedOnHeapWithinWorker, "bolt-3 shared on heap within worker")).shuffleGrouping("bolt-2");
+
+            StormTopology stormTopology = builder.createTopology();
+
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 500, 2000);
+            Config conf = createClusterConfig(strategyClass, cpuPercent, memoryOnHeap, memoryOffHeap, null);
+
+            conf.put(Config.TOPOLOGY_PRIORITY, 0);
+            conf.put(Config.TOPOLOGY_NAME, "testTopology");
+            conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
+            switch (schedulingLimitation) {
+                case WORKER_RESTRICTION_ONE_EXECUTOR:
+                    conf.put(Config.TOPOLOGY_RAS_ONE_EXECUTOR_PER_WORKER, true);
+                    break;
+                case WORKER_RESTRICTION_ONE_COMPONENT:
+                    conf.put(Config.TOPOLOGY_RAS_ONE_COMPONENT_PER_WORKER, true);
+                    break;
+            }
+            TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormTopology, 0,
+                    genExecsAndComps(stormTopology), CURRENT_TIME, "user");
+
+            Topologies topologies = new Topologies(topo);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
+
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(conf, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+
+            // [3,3] [7,7], [0,0] [2,2] [6,6] [1,1] [5,5] [4,4] sorted executor ordering
+            // spout  [0,0] [1,1]
+            // bolt-1 [2,2] [3,3]
+            // bolt-2 [6,6] [7,7]
+            // bolt-3 [4,4] [5,5]
+
+            // WorkerRestrictionType.WORKER_RESTRICTION_NONE
+            // expect 1 worker, 1 node
+
+            // WorkerRestrictionType.WORKER_RESTRICTION_ONE_EXECUTOR
+            // expect 8 workers, 2 nodes
+            // node r000s000 workers: bolt-1 bolt-2 spout bolt-1 (no memory sharing)
+            // node r000s001 workers: bolt-2 spout bolt-3 bolt-3 (no memory sharing)
+
+            // WorkerRestrictionType.WORKER_RESTRICTION_ONE_COMPONENT
+            // expect 4 workers, 1 node
+
+            for (Entry<String, SupervisorResources> entry : cluster.getSupervisorsResourcesMap().entrySet()) {
+                String supervisorId = entry.getKey();
+                SupervisorResources resources = entry.getValue();
+                assertTrue(resources.getTotalCpu() >= resources.getUsedCpu(), supervisorId);
+                assertTrue(resources.getTotalMem() >= resources.getUsedMem(), supervisorId);
+            }
 
-        int totalNumberOfTasks = spoutParallelism + boltParallelism * numBolts;
-        SchedulerAssignment assignment = cluster.getAssignmentById(topo.getId());
-        TopologyResources topologyResources = cluster.getTopologyResourcesMap().get(topo.getId());
-        long numNodes = assignment.getSlotToExecutors().keySet().stream().map(WorkerSlot::getNodeId).distinct().count();
-        String assignmentString = "Assignments:\n\t" + assignment.getSlotToExecutors().entrySet().stream()
-                .map(x -> String.format("Node=%s, components=%s",
-                        x.getKey().getNodeId(),
-                        x.getValue().stream()
-                                .map(topo::getComponentFromExecutor)
-                                .collect(Collectors.joining(","))
-                        )
-                )
-                .collect(Collectors.joining("\n\t"));
-
-        if (schedulingLimitation == WorkerRestrictionType.WORKER_RESTRICTION_NONE) {
-            // Everything should fit in a single slot
-            double totalExpectedCPU = totalNumberOfTasks * cpuPercent;
-            double totalExpectedOnHeap = (totalNumberOfTasks * memoryOnHeap) + sharedOnHeapWithinWorker;
-            double totalExpectedWorkerOffHeap = (totalNumberOfTasks * memoryOffHeap) + sharedOffHeapWithinWorker;
-
-            assertThat(assignment.getSlots().size(), is(1));
-            WorkerSlot ws = assignment.getSlots().iterator().next();
-            String nodeId = ws.getNodeId();
-            assertThat(assignment.getNodeIdToTotalSharedOffHeapNodeMemory().size(), is(1));
-            assertThat(assignment.getNodeIdToTotalSharedOffHeapNodeMemory().get(nodeId), closeTo(sharedOffHeapWithinNode, 0.01));
-            assertThat(assignment.getScheduledResources().size(), is(1));
-            WorkerResources resources = assignment.getScheduledResources().get(ws);
-            assertThat(resources.get_cpu(), closeTo(totalExpectedCPU, 0.01));
-            assertThat(resources.get_mem_on_heap(), closeTo(totalExpectedOnHeap, 0.01));
-            assertThat(resources.get_mem_off_heap(), closeTo(totalExpectedWorkerOffHeap, 0.01));
-            assertThat(resources.get_shared_mem_on_heap(), closeTo(sharedOnHeapWithinWorker, 0.01));
-            assertThat(resources.get_shared_mem_off_heap(), closeTo(sharedOffHeapWithinWorker, 0.01));
-        } else if (schedulingLimitation == WorkerRestrictionType.WORKER_RESTRICTION_ONE_EXECUTOR) {
-            double expectedMemOnHeap = (totalNumberOfTasks * memoryOnHeap) + 2 * sharedOnHeapWithinWorker;
-            double expectedMemOffHeap = (totalNumberOfTasks * memoryOffHeap) + 2 * sharedOffHeapWithinWorker + 2 * sharedOffHeapWithinNode;
-            double expectedMemSharedOnHeap = 2 * sharedOnHeapWithinWorker;
-            double expectedMemSharedOffHeap = 2 * sharedOffHeapWithinWorker + 2 * sharedOffHeapWithinNode;
-            double expectedMemNonSharedOnHeap = totalNumberOfTasks * memoryOnHeap;
-            double expectedMemNonSharedOffHeap = totalNumberOfTasks * memoryOffHeap;
-            assertThat(topologyResources.getAssignedMemOnHeap(), closeTo(expectedMemOnHeap, 0.01));
-            assertThat(topologyResources.getAssignedMemOffHeap(), closeTo(expectedMemOffHeap, 0.01));
-            assertThat(topologyResources.getAssignedSharedMemOnHeap(), closeTo(expectedMemSharedOnHeap, 0.01));
-            assertThat(topologyResources.getAssignedSharedMemOffHeap(), closeTo(expectedMemSharedOffHeap, 0.01));
-            assertThat(topologyResources.getAssignedNonSharedMemOnHeap(), closeTo(expectedMemNonSharedOnHeap, 0.01));
-            assertThat(topologyResources.getAssignedNonSharedMemOffHeap(), closeTo(expectedMemNonSharedOffHeap, 0.01));
-
-            double totalExpectedCPU = totalNumberOfTasks * cpuPercent;
-            assertThat(topologyResources.getAssignedCpu(), closeTo(totalExpectedCPU, 0.01));
-            int numAssignedWorkers = cluster.getAssignedNumWorkers(topo);
-            assertThat(numAssignedWorkers, is(8));
-            assertThat(assignment.getSlots().size(), is(8));
-            assertThat(assignmentString, numNodes, is(2L));
-        } else if (schedulingLimitation == WorkerRestrictionType.WORKER_RESTRICTION_ONE_COMPONENT) {
-            double expectedMemOnHeap = (totalNumberOfTasks * memoryOnHeap) + sharedOnHeapWithinWorker;
-            double expectedMemOffHeap = (totalNumberOfTasks * memoryOffHeap) + sharedOffHeapWithinWorker + sharedOffHeapWithinNode;
-            double expectedMemSharedOnHeap = sharedOnHeapWithinWorker;
-            double expectedMemSharedOffHeap = sharedOffHeapWithinWorker + sharedOffHeapWithinNode;
-            double expectedMemNonSharedOnHeap = totalNumberOfTasks * memoryOnHeap;
-            double expectedMemNonSharedOffHeap = totalNumberOfTasks * memoryOffHeap;
-            assertThat(topologyResources.getAssignedMemOnHeap(), closeTo(expectedMemOnHeap, 0.01));
-            assertThat(topologyResources.getAssignedMemOffHeap(), closeTo(expectedMemOffHeap, 0.01));
-            assertThat(topologyResources.getAssignedSharedMemOnHeap(), closeTo(expectedMemSharedOnHeap, 0.01));
-            assertThat(topologyResources.getAssignedSharedMemOffHeap(), closeTo(expectedMemSharedOffHeap, 0.01));
-            assertThat(topologyResources.getAssignedNonSharedMemOnHeap(), closeTo(expectedMemNonSharedOnHeap, 0.01));
-            assertThat(topologyResources.getAssignedNonSharedMemOffHeap(), closeTo(expectedMemNonSharedOffHeap, 0.01));
-
-            double totalExpectedCPU = totalNumberOfTasks * cpuPercent;
-            assertThat(topologyResources.getAssignedCpu(), closeTo(totalExpectedCPU, 0.01));
-            int numAssignedWorkers = cluster.getAssignedNumWorkers(topo);
-            assertThat(numAssignedWorkers, is(4));
-            assertThat(assignment.getSlots().size(), is(4));
-            assertThat(numNodes, is(1L));
+            int totalNumberOfTasks = spoutParallelism + boltParallelism * numBolts;
+            SchedulerAssignment assignment = cluster.getAssignmentById(topo.getId());
+            TopologyResources topologyResources = cluster.getTopologyResourcesMap().get(topo.getId());
+            long numNodes = assignment.getSlotToExecutors().keySet().stream().map(WorkerSlot::getNodeId).distinct().count();
+            String assignmentString = "Assignments:\n\t" + assignment.getSlotToExecutors().entrySet().stream()
+                    .map(x -> String.format("Node=%s, components=%s",
+                                    x.getKey().getNodeId(),
+                                    x.getValue().stream()
+                                            .map(topo::getComponentFromExecutor)
+                                            .collect(Collectors.joining(","))
+                            )
+                    )
+                    .collect(Collectors.joining("\n\t"));
+
+            if (schedulingLimitation == WorkerRestrictionType.WORKER_RESTRICTION_NONE) {
+                // Everything should fit in a single slot
+                double totalExpectedCPU = totalNumberOfTasks * cpuPercent;
+                double totalExpectedOnHeap = (totalNumberOfTasks * memoryOnHeap) + sharedOnHeapWithinWorker;
+                double totalExpectedWorkerOffHeap = (totalNumberOfTasks * memoryOffHeap) + sharedOffHeapWithinWorker;
+
+                assertThat(assignment.getSlots().size(), is(1));
+                WorkerSlot ws = assignment.getSlots().iterator().next();
+                String nodeId = ws.getNodeId();
+                assertThat(assignment.getNodeIdToTotalSharedOffHeapNodeMemory().size(), is(1));
+                assertThat(assignment.getNodeIdToTotalSharedOffHeapNodeMemory().get(nodeId), closeTo(sharedOffHeapWithinNode, 0.01));
+                assertThat(assignment.getScheduledResources().size(), is(1));
+                WorkerResources resources = assignment.getScheduledResources().get(ws);
+                assertThat(resources.get_cpu(), closeTo(totalExpectedCPU, 0.01));
+                assertThat(resources.get_mem_on_heap(), closeTo(totalExpectedOnHeap, 0.01));
+                assertThat(resources.get_mem_off_heap(), closeTo(totalExpectedWorkerOffHeap, 0.01));
+                assertThat(resources.get_shared_mem_on_heap(), closeTo(sharedOnHeapWithinWorker, 0.01));
+                assertThat(resources.get_shared_mem_off_heap(), closeTo(sharedOffHeapWithinWorker, 0.01));
+            } else if (schedulingLimitation == WorkerRestrictionType.WORKER_RESTRICTION_ONE_EXECUTOR) {
+                double expectedMemOnHeap = (totalNumberOfTasks * memoryOnHeap) + 2 * sharedOnHeapWithinWorker;
+                double expectedMemOffHeap = (totalNumberOfTasks * memoryOffHeap) + 2 * sharedOffHeapWithinWorker + 2 * sharedOffHeapWithinNode;
+                double expectedMemSharedOnHeap = 2 * sharedOnHeapWithinWorker;
+                double expectedMemSharedOffHeap = 2 * sharedOffHeapWithinWorker + 2 * sharedOffHeapWithinNode;
+                double expectedMemNonSharedOnHeap = totalNumberOfTasks * memoryOnHeap;
+                double expectedMemNonSharedOffHeap = totalNumberOfTasks * memoryOffHeap;
+                assertThat(topologyResources.getAssignedMemOnHeap(), closeTo(expectedMemOnHeap, 0.01));
+                assertThat(topologyResources.getAssignedMemOffHeap(), closeTo(expectedMemOffHeap, 0.01));
+                assertThat(topologyResources.getAssignedSharedMemOnHeap(), closeTo(expectedMemSharedOnHeap, 0.01));
+                assertThat(topologyResources.getAssignedSharedMemOffHeap(), closeTo(expectedMemSharedOffHeap, 0.01));
+                assertThat(topologyResources.getAssignedNonSharedMemOnHeap(), closeTo(expectedMemNonSharedOnHeap, 0.01));
+                assertThat(topologyResources.getAssignedNonSharedMemOffHeap(), closeTo(expectedMemNonSharedOffHeap, 0.01));
+
+                double totalExpectedCPU = totalNumberOfTasks * cpuPercent;
+                assertThat(topologyResources.getAssignedCpu(), closeTo(totalExpectedCPU, 0.01));
+                int numAssignedWorkers = cluster.getAssignedNumWorkers(topo);
+                assertThat(numAssignedWorkers, is(8));
+                assertThat(assignment.getSlots().size(), is(8));
+                assertThat(assignmentString, numNodes, is(2L));
+            } else if (schedulingLimitation == WorkerRestrictionType.WORKER_RESTRICTION_ONE_COMPONENT) {
+                double expectedMemOnHeap = (totalNumberOfTasks * memoryOnHeap) + sharedOnHeapWithinWorker;
+                double expectedMemOffHeap = (totalNumberOfTasks * memoryOffHeap) + sharedOffHeapWithinWorker + sharedOffHeapWithinNode;
+                double expectedMemSharedOnHeap = sharedOnHeapWithinWorker;
+                double expectedMemSharedOffHeap = sharedOffHeapWithinWorker + sharedOffHeapWithinNode;
+                double expectedMemNonSharedOnHeap = totalNumberOfTasks * memoryOnHeap;
+                double expectedMemNonSharedOffHeap = totalNumberOfTasks * memoryOffHeap;
+                assertThat(topologyResources.getAssignedMemOnHeap(), closeTo(expectedMemOnHeap, 0.01));
+                assertThat(topologyResources.getAssignedMemOffHeap(), closeTo(expectedMemOffHeap, 0.01));
+                assertThat(topologyResources.getAssignedSharedMemOnHeap(), closeTo(expectedMemSharedOnHeap, 0.01));
+                assertThat(topologyResources.getAssignedSharedMemOffHeap(), closeTo(expectedMemSharedOffHeap, 0.01));
+                assertThat(topologyResources.getAssignedNonSharedMemOnHeap(), closeTo(expectedMemNonSharedOnHeap, 0.01));
+                assertThat(topologyResources.getAssignedNonSharedMemOffHeap(), closeTo(expectedMemNonSharedOffHeap, 0.01));
+
+                double totalExpectedCPU = totalNumberOfTasks * cpuPercent;
+                assertThat(topologyResources.getAssignedCpu(), closeTo(totalExpectedCPU, 0.01));
+                int numAssignedWorkers = cluster.getAssignedNumWorkers(topo);
+                assertThat(numAssignedWorkers, is(4));
+                assertThat(assignment.getSlots().size(), is(4));
+                assertThat(numNodes, is(1L));
+            }
         }
     }
 
@@ -468,109 +474,111 @@ public class TestDefaultResourceAwareStrategy {
     @ParameterizedTest
     @ValueSource(ints = {-1, 0, 1, 2})
     public void testDefaultResourceAwareStrategyWithoutSettingAckerExecutors(int numOfAckersPerWorker)
-        throws InvalidTopologyException {
-        int spoutParallelism = 1;
-        int boltParallelism = 2;
-        TopologyBuilder builder = new TopologyBuilder();
-        builder.setSpout("spout", new TestSpout(),
-            spoutParallelism);
-        builder.setBolt("bolt-1", new TestBolt(),
-            boltParallelism).shuffleGrouping("spout");
-        builder.setBolt("bolt-2", new TestBolt(),
-            boltParallelism).shuffleGrouping("bolt-1");
-        builder.setBolt("bolt-3", new TestBolt(),
-            boltParallelism).shuffleGrouping("bolt-2");
-
-        String topoName = "testTopology";
-
-        StormTopology stormTopology = builder.createTopology();
-
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 200, 2000);
-        Config conf = createClusterConfig(50, 450, 0, null);
-        conf.put(Config.TOPOLOGY_PRIORITY, 0);
-        conf.put(Config.TOPOLOGY_NAME, topoName);
-        conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
-        conf.put(Config.TOPOLOGY_SUBMITTER_USER, "user");
-
-        // Topology needs 2 workers (estimated by nimbus based on resources),
-        // but with ackers added, probably more worker will be launched.
-        // Parameterized test on different numOfAckersPerWorker
-        if (numOfAckersPerWorker == -1) {
-            // Both Config.TOPOLOGY_ACKER_EXECUTORS and Config.TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER are not set
-            // Default will be 2 (estimate num of workers) and 1 respectively
-        } else {
-            conf.put(Config.TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER, numOfAckersPerWorker);
-        }
-
-        int estimatedNumWorker = ServerUtils.getEstimatedWorkerCountForRasTopo(conf, stormTopology);
-        Nimbus.setUpAckerExecutorConfigs(topoName, conf, conf, estimatedNumWorker);
-
-        conf.put(Config.TOPOLOGY_ACKER_RESOURCES_ONHEAP_MEMORY_MB, 250);
-        conf.put(Config.TOPOLOGY_ACKER_CPU_PCORE_PERCENT, 50);
-
-        TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormTopology, 0,
-            genExecsAndComps(StormCommon.systemTopology(conf, stormTopology)), CURRENT_TIME, "user");
-
-        Topologies topologies = new Topologies(topo);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
-
-        scheduler = new ResourceAwareScheduler();
+            throws InvalidTopologyException {
+        for (Class strategyClass: strategyClasses) {
+            int spoutParallelism = 1;
+            int boltParallelism = 2;
+            TopologyBuilder builder = new TopologyBuilder();
+            builder.setSpout("spout", new TestSpout(),
+                    spoutParallelism);
+            builder.setBolt("bolt-1", new TestBolt(),
+                    boltParallelism).shuffleGrouping("spout");
+            builder.setBolt("bolt-2", new TestBolt(),
+                    boltParallelism).shuffleGrouping("bolt-1");
+            builder.setBolt("bolt-3", new TestBolt(),
+                    boltParallelism).shuffleGrouping("bolt-2");
+
+            String topoName = "testTopology";
+
+            StormTopology stormTopology = builder.createTopology();
+
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 200, 2000);
+            Config conf = createClusterConfig(strategyClass, 50, 450, 0, null);
+            conf.put(Config.TOPOLOGY_PRIORITY, 0);
+            conf.put(Config.TOPOLOGY_NAME, topoName);
+            conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
+            conf.put(Config.TOPOLOGY_SUBMITTER_USER, "user");
+
+            // Topology needs 2 workers (estimated by nimbus based on resources),
+            // but with ackers added, probably more worker will be launched.
+            // Parameterized test on different numOfAckersPerWorker
+            if (numOfAckersPerWorker == -1) {
+                // Both Config.TOPOLOGY_ACKER_EXECUTORS and Config.TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER are not set
+                // Default will be 2 (estimate num of workers) and 1 respectively
+            } else {
+                conf.put(Config.TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER, numOfAckersPerWorker);
+            }
 
-        scheduler.prepare(conf, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
+            int estimatedNumWorker = ServerUtils.getEstimatedWorkerCountForRasTopo(conf, stormTopology);
+            Nimbus.setUpAckerExecutorConfigs(topoName, conf, conf, estimatedNumWorker);
+
+            conf.put(Config.TOPOLOGY_ACKER_RESOURCES_ONHEAP_MEMORY_MB, 250);
+            conf.put(Config.TOPOLOGY_ACKER_CPU_PCORE_PERCENT, 50);
+
+            TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormTopology, 0,
+                    genExecsAndComps(StormCommon.systemTopology(conf, stormTopology)), CURRENT_TIME, "user");
+
+            Topologies topologies = new Topologies(topo);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
+
+            scheduler = new ResourceAwareScheduler();
+
+            scheduler.prepare(conf, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+
+            // Ordered execs: [[6, 6], [2, 2], [4, 4], [5, 5], [1, 1], [3, 3], [0, 0], [8, 8], [7, 7]]
+            // Ackers: [[8, 8], [7, 7]] (+ [[9, 9], [10, 10]] when numOfAckersPerWorker=2)
+            HashSet<HashSet<ExecutorDetails>> expectedScheduling = new HashSet<>();
+            if (numOfAckersPerWorker == -1 || numOfAckersPerWorker == 1) {
+                // Setting topology.acker.executors = null and topology.acker.executors.per.worker = null
+                // are equivalent to topology.acker.executors = null and topology.acker.executors.per.worker = 1
+                expectedScheduling.add(new HashSet<>(Arrays.asList(
+                        new ExecutorDetails(6, 6), //bolt-3
+                        new ExecutorDetails(2, 2), //bolt-1
+                        new ExecutorDetails(4, 4), //bolt-2
+                        new ExecutorDetails(8, 8)))); //acker
+                expectedScheduling.add(new HashSet<>(Arrays.asList(
+                        new ExecutorDetails(5, 5), //bolt-3
+                        new ExecutorDetails(1, 1), //bolt-1
+                        new ExecutorDetails(3, 3), //bolt-2
+                        new ExecutorDetails(7, 7)))); //acker
+                expectedScheduling.add(new HashSet<>(Arrays.asList(
+                        new ExecutorDetails(0, 0)))); //spout
+            } else if (numOfAckersPerWorker == 0) {
+                expectedScheduling.add(new HashSet<>(Arrays.asList(
+                        new ExecutorDetails(6, 6), //bolt-3
+                        new ExecutorDetails(2, 2), //bolt-1
+                        new ExecutorDetails(4, 4), //bolt-2
+                        new ExecutorDetails(5, 5)))); //bolt-3
+                expectedScheduling.add(new HashSet<>(Arrays.asList(
+                        new ExecutorDetails(0, 0), //spout
+                        new ExecutorDetails(3, 3), //bolt-2
+                        new ExecutorDetails(1, 1)))); //bolt-1
+            } else if (numOfAckersPerWorker == 2) {
+                expectedScheduling.add(new HashSet<>(Arrays.asList(
+                        new ExecutorDetails(6, 6), //bolt-3
+                        new ExecutorDetails(2, 2), //bolt-1
+                        new ExecutorDetails(7, 7), //acker
+                        new ExecutorDetails(8, 8)))); //acker
+                expectedScheduling.add(new HashSet<>(Arrays.asList(
+                        new ExecutorDetails(4, 4), //bolt-2
+                        new ExecutorDetails(5, 5), //bolt-3
+                        new ExecutorDetails(9, 9), //acker
+                        new ExecutorDetails(10, 10)))); //acker
+                expectedScheduling.add(new HashSet<>(Arrays.asList(
+                        new ExecutorDetails(1, 1), //bolt-1
+                        new ExecutorDetails(3, 3), //bolt-2
+                        new ExecutorDetails(0, 0)))); //spout
+            }
+            HashSet<HashSet<ExecutorDetails>> foundScheduling = new HashSet<>();
+            SchedulerAssignment assignment = cluster.getAssignmentById("testTopology-id");
+            for (Collection<ExecutorDetails> execs : assignment.getSlotToExecutors().values()) {
+                foundScheduling.add(new HashSet<>(execs));
+            }
 
-        // Ordered execs: [[6, 6], [2, 2], [4, 4], [5, 5], [1, 1], [3, 3], [0, 0], [8, 8], [7, 7]]
-        // Ackers: [[8, 8], [7, 7]] (+ [[9, 9], [10, 10]] when numOfAckersPerWorker=2)
-        HashSet<HashSet<ExecutorDetails>> expectedScheduling = new HashSet<>();
-        if (numOfAckersPerWorker == -1 || numOfAckersPerWorker == 1) {
-            // Setting topology.acker.executors = null and topology.acker.executors.per.worker = null
-            // are equivalent to topology.acker.executors = null and topology.acker.executors.per.worker = 1
-            expectedScheduling.add(new HashSet<>(Arrays.asList(
-                new ExecutorDetails(6, 6), //bolt-3
-                new ExecutorDetails(2, 2), //bolt-1
-                new ExecutorDetails(4, 4), //bolt-2
-                new ExecutorDetails(8, 8)))); //acker
-            expectedScheduling.add(new HashSet<>(Arrays.asList(
-                new ExecutorDetails(5, 5), //bolt-3
-                new ExecutorDetails(1, 1), //bolt-1
-                new ExecutorDetails(3, 3), //bolt-2
-                new ExecutorDetails(7, 7)))); //acker
-            expectedScheduling.add(new HashSet<>(Arrays.asList(
-                new ExecutorDetails(0, 0)))); //spout
-        } else if (numOfAckersPerWorker == 0) {
-            expectedScheduling.add(new HashSet<>(Arrays.asList(
-                new ExecutorDetails(6, 6), //bolt-3
-                new ExecutorDetails(2, 2), //bolt-1
-                new ExecutorDetails(4, 4), //bolt-2
-                new ExecutorDetails(5, 5)))); //bolt-3
-            expectedScheduling.add(new HashSet<>(Arrays.asList(
-                new ExecutorDetails(0, 0), //spout
-                new ExecutorDetails(3, 3), //bolt-2
-                new ExecutorDetails(1, 1)))); //bolt-1
-        } else if (numOfAckersPerWorker == 2) {
-            expectedScheduling.add(new HashSet<>(Arrays.asList(
-                new ExecutorDetails(6, 6), //bolt-3
-                new ExecutorDetails(2, 2), //bolt-1
-                new ExecutorDetails(7, 7), //acker
-                new ExecutorDetails(8, 8)))); //acker
-            expectedScheduling.add(new HashSet<>(Arrays.asList(
-                new ExecutorDetails(4, 4), //bolt-2
-                new ExecutorDetails(5, 5), //bolt-3
-                new ExecutorDetails(9, 9), //acker
-                new ExecutorDetails(10, 10)))); //acker
-            expectedScheduling.add(new HashSet<>(Arrays.asList(
-                new ExecutorDetails(1, 1), //bolt-1
-                new ExecutorDetails(3, 3), //bolt-2
-                new ExecutorDetails(0, 0)))); //spout
+            assertEquals(expectedScheduling, foundScheduling);
         }
-        HashSet<HashSet<ExecutorDetails>> foundScheduling = new HashSet<>();
-        SchedulerAssignment assignment = cluster.getAssignmentById("testTopology-id");
-        for (Collection<ExecutorDetails> execs : assignment.getSlotToExecutors().values()) {
-            foundScheduling.add(new HashSet<>(execs));
-        }
-
-        assertEquals(expectedScheduling, foundScheduling);
     }
 
     /**
@@ -583,85 +591,86 @@ public class TestDefaultResourceAwareStrategy {
     @ParameterizedTest
     @ValueSource(ints = {-1, 0, 2, 300})
     public void testDefaultResourceAwareStrategyWithSettingAckerExecutors(int numOfAckersPerWorker)
-        throws InvalidTopologyException {
-
-        int spoutParallelism = 1;
-        int boltParallelism = 2;
-        TopologyBuilder builder = new TopologyBuilder();
-        builder.setSpout("spout", new TestSpout(),
-            spoutParallelism);
-        builder.setBolt("bolt-1", new TestBolt(),
-            boltParallelism).shuffleGrouping("spout");
-        builder.setBolt("bolt-2", new TestBolt(),
-            boltParallelism).shuffleGrouping("bolt-1");
-        builder.setBolt("bolt-3", new TestBolt(),
-            boltParallelism).shuffleGrouping("bolt-2");
-
-        String topoName = "testTopology";
-
-        StormTopology stormTopology = builder.createTopology();
-
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 200, 2000);
-        Config conf = createClusterConfig(50, 450, 0, null);
-        conf.put(Config.TOPOLOGY_PRIORITY, 0);
-        conf.put(Config.TOPOLOGY_NAME, topoName);
-        conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
-        conf.put(Config.TOPOLOGY_SUBMITTER_USER, "user");
-
-        conf.put(Config.TOPOLOGY_ACKER_EXECUTORS, 4);
-        conf.put(Config.TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER, numOfAckersPerWorker);
-
-
-        if (numOfAckersPerWorker == -1) {
-            // Leave topology.acker.executors.per.worker unset
-        } else {
+            throws InvalidTopologyException {
+        for (Class strategyClass: strategyClasses) {
+            int spoutParallelism = 1;
+            int boltParallelism = 2;
+            TopologyBuilder builder = new TopologyBuilder();
+            builder.setSpout("spout", new TestSpout(),
+                    spoutParallelism);
+            builder.setBolt("bolt-1", new TestBolt(),
+                    boltParallelism).shuffleGrouping("spout");
+            builder.setBolt("bolt-2", new TestBolt(),
+                    boltParallelism).shuffleGrouping("bolt-1");
+            builder.setBolt("bolt-3", new TestBolt(),
+                    boltParallelism).shuffleGrouping("bolt-2");
+
+            String topoName = "testTopology";
+
+            StormTopology stormTopology = builder.createTopology();
+
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 200, 2000);
+            Config conf = createClusterConfig(strategyClass, 50, 450, 0, null);
+            conf.put(Config.TOPOLOGY_PRIORITY, 0);
+            conf.put(Config.TOPOLOGY_NAME, topoName);
+            conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
+            conf.put(Config.TOPOLOGY_SUBMITTER_USER, "user");
+
+            conf.put(Config.TOPOLOGY_ACKER_EXECUTORS, 4);
             conf.put(Config.TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER, numOfAckersPerWorker);
-        }
 
-        int estimatedNumWorker = ServerUtils.getEstimatedWorkerCountForRasTopo(conf, stormTopology);
-        Nimbus.setUpAckerExecutorConfigs(topoName, conf, conf, estimatedNumWorker);
-
-        conf.put(Config.TOPOLOGY_ACKER_RESOURCES_ONHEAP_MEMORY_MB, 250);
-        conf.put(Config.TOPOLOGY_ACKER_CPU_PCORE_PERCENT, 50);
-
-        TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormTopology, 0,
-            genExecsAndComps(StormCommon.systemTopology(conf, stormTopology)), CURRENT_TIME, "user");
-
-        Topologies topologies = new Topologies(topo);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
-
-        scheduler = new ResourceAwareScheduler();
-
-        scheduler.prepare(conf, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-
-        // Sorted execs: [[6, 6], [2, 2], [4, 4], [5, 5], [1, 1], [3, 3], [0, 0], [8, 8], [7, 7], [10, 10], [9, 9]]
-        // Ackers: [[8, 8], [7, 7], [10, 10], [9, 9]]
-
-        HashSet<HashSet<ExecutorDetails>> expectedScheduling = new HashSet<>();
-        expectedScheduling.add(new HashSet<>(Arrays.asList(
-            new ExecutorDetails(6, 6), //bolt-3
-            new ExecutorDetails(2, 2), //bolt-1
-            new ExecutorDetails(7, 7), //acker
-            new ExecutorDetails(8, 8)))); //acker
-        expectedScheduling.add(new HashSet<>(Arrays.asList(
-            new ExecutorDetails(5, 5), //bolt-3
-            new ExecutorDetails(4, 4), //bolt-2
-            new ExecutorDetails(9, 9), //acker
-            new ExecutorDetails(10, 10)))); //acker
-        expectedScheduling.add(new HashSet<>(Arrays.asList(
-            new ExecutorDetails(0, 0), //spout
-            new ExecutorDetails(3, 3), //bolt-2
-            new ExecutorDetails(1, 1)))); //bolt-1
-
-        HashSet<HashSet<ExecutorDetails>> foundScheduling = new HashSet<>();
-        SchedulerAssignment assignment = cluster.getAssignmentById("testTopology-id");
-        for (Collection<ExecutorDetails> execs : assignment.getSlotToExecutors().values()) {
-            foundScheduling.add(new HashSet<>(execs));
-        }
 
-        assertEquals(expectedScheduling, foundScheduling);
+            if (numOfAckersPerWorker == -1) {
+                // Leave topology.acker.executors.per.worker unset
+            } else {
+                conf.put(Config.TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER, numOfAckersPerWorker);
+            }
+
+            int estimatedNumWorker = ServerUtils.getEstimatedWorkerCountForRasTopo(conf, stormTopology);
+            Nimbus.setUpAckerExecutorConfigs(topoName, conf, conf, estimatedNumWorker);
+
+            conf.put(Config.TOPOLOGY_ACKER_RESOURCES_ONHEAP_MEMORY_MB, 250);
+            conf.put(Config.TOPOLOGY_ACKER_CPU_PCORE_PERCENT, 50);
+
+            TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormTopology, 0,
+                    genExecsAndComps(StormCommon.systemTopology(conf, stormTopology)), CURRENT_TIME, "user");
+
+            Topologies topologies = new Topologies(topo);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
+
+            scheduler = new ResourceAwareScheduler();
+
+            scheduler.prepare(conf, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+
+            // Sorted execs: [[6, 6], [2, 2], [4, 4], [5, 5], [1, 1], [3, 3], [0, 0], [8, 8], [7, 7], [10, 10], [9, 9]]
+            // Ackers: [[8, 8], [7, 7], [10, 10], [9, 9]]
+
+            HashSet<HashSet<ExecutorDetails>> expectedScheduling = new HashSet<>();
+            expectedScheduling.add(new HashSet<>(Arrays.asList(
+                    new ExecutorDetails(6, 6), //bolt-3
+                    new ExecutorDetails(2, 2), //bolt-1
+                    new ExecutorDetails(7, 7), //acker
+                    new ExecutorDetails(8, 8)))); //acker
+            expectedScheduling.add(new HashSet<>(Arrays.asList(
+                    new ExecutorDetails(5, 5), //bolt-3
+                    new ExecutorDetails(4, 4), //bolt-2
+                    new ExecutorDetails(9, 9), //acker
+                    new ExecutorDetails(10, 10)))); //acker
+            expectedScheduling.add(new HashSet<>(Arrays.asList(
+                    new ExecutorDetails(0, 0), //spout
+                    new ExecutorDetails(3, 3), //bolt-2
+                    new ExecutorDetails(1, 1)))); //bolt-1
+
+            HashSet<HashSet<ExecutorDetails>> foundScheduling = new HashSet<>();
+            SchedulerAssignment assignment = cluster.getAssignmentById("testTopology-id");
+            for (Collection<ExecutorDetails> execs : assignment.getSlotToExecutors().values()) {
+                foundScheduling.add(new HashSet<>(execs));
+            }
+
+            assertEquals(expectedScheduling, foundScheduling);
+        }
     }
 
     /**
@@ -669,61 +678,63 @@ public class TestDefaultResourceAwareStrategy {
      */
     @Test
     public void testDefaultResourceAwareStrategyInFavorOfShuffle()
-        throws InvalidTopologyException {
-        int spoutParallelism = 1;
-        int boltParallelism = 2;
-        TopologyBuilder builder = new TopologyBuilder();
-        builder.setSpout("spout", new TestSpout(),
-            spoutParallelism);
-        builder.setBolt("bolt-1", new TestBolt(),
-            boltParallelism).shuffleGrouping("spout");
-        builder.setBolt("bolt-2", new TestBolt(),
-            boltParallelism).shuffleGrouping("bolt-1");
-        builder.setBolt("bolt-3", new TestBolt(),
-            boltParallelism).shuffleGrouping("bolt-2");
-
-        StormTopology stormToplogy = builder.createTopology();
-
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 200, 2000);
-        Config conf = createClusterConfig(50, 250, 250, null);
-        conf.put(Config.TOPOLOGY_PRIORITY, 0);
-        conf.put(Config.TOPOLOGY_NAME, "testTopology");
-        conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE);
-        conf.put(Config.TOPOLOGY_SUBMITTER_USER, "user");
-        conf.put(Config.TOPOLOGY_RAS_ORDER_EXECUTORS_BY_PROXIMITY_NEEDS, true);
-
-        TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormToplogy, 0,
-            genExecsAndComps(StormCommon.systemTopology(conf, stormToplogy)), CURRENT_TIME, "user");
-
-        Topologies topologies = new Topologies(topo);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
-
-        ResourceAwareScheduler rs = new ResourceAwareScheduler();
-
-        rs.prepare(conf, new StormMetricsRegistry());
-        rs.schedule(topologies, cluster);
-        // Sorted execs: [[0, 0], [2, 2], [6, 6], [4, 4], [1, 1], [5, 5], [3, 3], [7, 7]]
-        // Ackers: [[7, 7]]]
-
-        HashSet<HashSet<ExecutorDetails>> expectedScheduling = new HashSet<>();
-        expectedScheduling.add(new HashSet<>(Arrays.asList(
-            new ExecutorDetails(0, 0), //spout
-            new ExecutorDetails(6, 6), //bolt-2
-            new ExecutorDetails(2, 2), //bolt-1
-            new ExecutorDetails(7, 7)))); //acker
-        expectedScheduling.add(new HashSet<>(Arrays.asList(
-            new ExecutorDetails(3, 3), //bolt-3
-            new ExecutorDetails(5, 5), //bolt-2
-            new ExecutorDetails(4, 4), //bolt-3
-            new ExecutorDetails(1, 1)))); //bolt-1
-        HashSet<HashSet<ExecutorDetails>> foundScheduling = new HashSet<>();
-        SchedulerAssignment assignment = cluster.getAssignmentById("testTopology-id");
-        for (Collection<ExecutorDetails> execs : assignment.getSlotToExecutors().values()) {
-            foundScheduling.add(new HashSet<>(execs));
-        }
+            throws InvalidTopologyException {
+        for (Class strategyClass: strategyClasses) {
+            int spoutParallelism = 1;
+            int boltParallelism = 2;
+            TopologyBuilder builder = new TopologyBuilder();
+            builder.setSpout("spout", new TestSpout(),
+                    spoutParallelism);
+            builder.setBolt("bolt-1", new TestBolt(),
+                    boltParallelism).shuffleGrouping("spout");
+            builder.setBolt("bolt-2", new TestBolt(),
+                    boltParallelism).shuffleGrouping("bolt-1");
+            builder.setBolt("bolt-3", new TestBolt(),
+                    boltParallelism).shuffleGrouping("bolt-2");
+
+            StormTopology stormToplogy = builder.createTopology();
+
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 200, 2000);
+            Config conf = createClusterConfig(strategyClass, 50, 250, 250, null);
+            conf.put(Config.TOPOLOGY_PRIORITY, 0);
+            conf.put(Config.TOPOLOGY_NAME, "testTopology");
+            conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE);
+            conf.put(Config.TOPOLOGY_SUBMITTER_USER, "user");
+            conf.put(Config.TOPOLOGY_RAS_ORDER_EXECUTORS_BY_PROXIMITY_NEEDS, true);
+
+            TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormToplogy, 0,
+                    genExecsAndComps(StormCommon.systemTopology(conf, stormToplogy)), CURRENT_TIME, "user");
+
+            Topologies topologies = new Topologies(topo);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
+
+            ResourceAwareScheduler rs = new ResourceAwareScheduler();
+
+            rs.prepare(conf, new StormMetricsRegistry());
+            rs.schedule(topologies, cluster);
+            // Sorted execs: [[0, 0], [2, 2], [6, 6], [4, 4], [1, 1], [5, 5], [3, 3], [7, 7]]
+            // Ackers: [[7, 7]]]
+
+            HashSet<HashSet<ExecutorDetails>> expectedScheduling = new HashSet<>();
+            expectedScheduling.add(new HashSet<>(Arrays.asList(
+                    new ExecutorDetails(0, 0), //spout
+                    new ExecutorDetails(6, 6), //bolt-2
+                    new ExecutorDetails(2, 2), //bolt-1
+                    new ExecutorDetails(7, 7)))); //acker
+            expectedScheduling.add(new HashSet<>(Arrays.asList(
+                    new ExecutorDetails(3, 3), //bolt-3
+                    new ExecutorDetails(5, 5), //bolt-2
+                    new ExecutorDetails(4, 4), //bolt-3
+                    new ExecutorDetails(1, 1)))); //bolt-1
+            HashSet<HashSet<ExecutorDetails>> foundScheduling = new HashSet<>();
+            SchedulerAssignment assignment = cluster.getAssignmentById("testTopology-id");
+            for (Collection<ExecutorDetails> execs : assignment.getSlotToExecutors().values()) {
+                foundScheduling.add(new HashSet<>(execs));
+            }
 
-        assertEquals(expectedScheduling, foundScheduling);
+            assertEquals(expectedScheduling, foundScheduling);
+        }
     }
 
     /**
@@ -731,113 +742,115 @@ public class TestDefaultResourceAwareStrategy {
      */
     @Test
     public void testMultipleRacks() {
-        final Map<String, SupervisorDetails> supMap = new HashMap<>();
-        final Map<String, SupervisorDetails> supMapRack0 = genSupervisors(10, 4, 0, 400, 8000);
-        //generate another rack of supervisors with less resources
-        final Map<String, SupervisorDetails> supMapRack1 = genSupervisors(10, 4, 10, 200, 4000);
-
-        //generate some supervisors that are depleted of one resource
-        final Map<String, SupervisorDetails> supMapRack2 = genSupervisors(10, 4, 20, 0, 8000);
-
-        //generate some that has alot of memory but little of cpu
-        final Map<String, SupervisorDetails> supMapRack3 = genSupervisors(10, 4, 30, 10, 8000 * 2 + 4000);
-
-        //generate some that has alot of cpu but little of memory
-        final Map<String, SupervisorDetails> supMapRack4 = genSupervisors(10, 4, 40, 400 + 200 + 10, 1000);
-
-        //Generate some that have neither resource, to verify that the strategy will prioritize this last
-        //Also put a generic resource with 0 value in the resources list, to verify that it doesn't affect the sorting
-        final Map<String, SupervisorDetails> supMapRack5 = genSupervisors(10, 4, 50, 0.0, 0.0, Collections.singletonMap("gpu.count", 0.0));
-
-        supMap.putAll(supMapRack0);
-        supMap.putAll(supMapRack1);
-        supMap.putAll(supMapRack2);
-        supMap.putAll(supMapRack3);
-        supMap.putAll(supMapRack4);
-        supMap.putAll(supMapRack5);
-
-        Config config = createClusterConfig(100, 500, 500, null);
-        config.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE);
-        INimbus iNimbus = new INimbusTest();
-
-        //create test DNSToSwitchMapping plugin
-        DNSToSwitchMapping TestNetworkTopographyPlugin =
-            new TestDNSToSwitchMapping(supMapRack0, supMapRack1, supMapRack2, supMapRack3, supMapRack4, supMapRack5);
-
-        //generate topologies
-        TopologyDetails topo1 = genTopology("topo-1", config, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
-        TopologyDetails topo2 = genTopology("topo-2", config, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
-        
-        Topologies topologies = new Topologies(topo1, topo2);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-        
-        List<String> supHostnames = new LinkedList<>();
-        for (SupervisorDetails sup : supMap.values()) {
-            supHostnames.add(sup.getHost());
-        }
-        Map<String, List<String>> rackToNodes = new HashMap<>();
-        Map<String, String> resolvedSuperVisors =  TestNetworkTopographyPlugin.resolve(supHostnames);
-        for (Map.Entry<String, String> entry : resolvedSuperVisors.entrySet()) {
-            String hostName = entry.getKey();
-            String rack = entry.getValue();
-            rackToNodes.computeIfAbsent(rack, rid -> new ArrayList<>()).add(hostName);
-        }
-        cluster.setNetworkTopography(rackToNodes);
-
-        DefaultResourceAwareStrategyOld rs = new DefaultResourceAwareStrategyOld();
-        
-        rs.prepareForScheduling(cluster, topo1);
-        INodeSorter nodeSorter = new NodeSorterHostProximity(cluster, topo1, BaseResourceAwareStrategy.NodeSortType.DEFAULT_RAS);
-        nodeSorter.prepare(null);
-        Iterable<ObjectResourcesItem> sortedRacks = nodeSorter.getSortedRacks();
-
-        Iterator<ObjectResourcesItem> it = sortedRacks.iterator();
-        // Ranked first since rack-0 has the most balanced set of resources
-        assertEquals("rack-0", it.next().id, "rack-0 should be ordered first");
-        // Ranked second since rack-1 has a balanced set of resources but less than rack-0
-        assertEquals("rack-1", it.next().id, "rack-1 should be ordered second");
-        // Ranked third since rack-4 has a lot of cpu but not a lot of memory
-        assertEquals("rack-4", it.next().id, "rack-4 should be ordered third");
-        // Ranked fourth since rack-3 has alot of memory but not cpu
-        assertEquals("rack-3", it.next().id, "rack-3 should be ordered fourth");
-        //Ranked fifth since rack-2 has not cpu resources
-        assertEquals("rack-2", it.next().id, "rack-2 should be ordered fifth");
-        //Ranked last since rack-5 has neither CPU nor memory available
-        assertEquals("rack-5", it.next().id, "Rack-5 should be ordered sixth");
-
-        SchedulingResult schedulingResult = rs.schedule(cluster, topo1);
-        assert(schedulingResult.isSuccess());
-        SchedulerAssignment assignment = cluster.getAssignmentById(topo1.getId());
-        for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
-            //make sure all workers on scheduled in rack-0
-            assertEquals("rack-0",
-                    resolvedSuperVisors.get(rs.idToNode(ws.getNodeId()).getHostname()), "assert worker scheduled on rack-0");
-        }
-        assertEquals(0, cluster.getUnassignedExecutors(topo1).size(), "All executors in topo-1 scheduled");
-
-        //Test if topology is already partially scheduled on one rack
-        Iterator<ExecutorDetails> executorIterator = topo2.getExecutors().iterator();
-        List<String> nodeHostnames = rackToNodes.get("rack-1");
-        for (int i = 0; i< topo2.getExecutors().size()/2; i++) {
-            String nodeHostname = nodeHostnames.get(i % nodeHostnames.size());
-            RasNode node = rs.hostnameToNodes(nodeHostname).get(0);
-            WorkerSlot targetSlot = node.getFreeSlots().iterator().next();
-            ExecutorDetails targetExec = executorIterator.next();
-            // to keep track of free slots
-            node.assign(targetSlot, topo2, Collections.singletonList(targetExec));
-        }
+        for (Class strategyClass: strategyClasses) {
+            final Map<String, SupervisorDetails> supMap = new HashMap<>();
+            final Map<String, SupervisorDetails> supMapRack0 = genSupervisors(10, 4, 0, 400, 8000);
+            //generate another rack of supervisors with less resources
+            final Map<String, SupervisorDetails> supMapRack1 = genSupervisors(10, 4, 10, 200, 4000);
+
+            //generate some supervisors that are depleted of one resource
+            final Map<String, SupervisorDetails> supMapRack2 = genSupervisors(10, 4, 20, 0, 8000);
+
+            //generate some that has alot of memory but little of cpu
+            final Map<String, SupervisorDetails> supMapRack3 = genSupervisors(10, 4, 30, 10, 8000 * 2 + 4000);
+
+            //generate some that has alot of cpu but little of memory
+            final Map<String, SupervisorDetails> supMapRack4 = genSupervisors(10, 4, 40, 400 + 200 + 10, 1000);
+
+            //Generate some that have neither resource, to verify that the strategy will prioritize this last
+            //Also put a generic resource with 0 value in the resources list, to verify that it doesn't affect the sorting
+            final Map<String, SupervisorDetails> supMapRack5 = genSupervisors(10, 4, 50, 0.0, 0.0, Collections.singletonMap("gpu.count", 0.0));
+
+            supMap.putAll(supMapRack0);
+            supMap.putAll(supMapRack1);
+            supMap.putAll(supMapRack2);
+            supMap.putAll(supMapRack3);
+            supMap.putAll(supMapRack4);
+            supMap.putAll(supMapRack5);
+
+            Config config = createClusterConfig(strategyClass, 100, 500, 500, null);
+            config.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE);
+            INimbus iNimbus = new INimbusTest();
+
+            //create test DNSToSwitchMapping plugin
+            DNSToSwitchMapping TestNetworkTopographyPlugin =
+                    new TestDNSToSwitchMapping(supMapRack0, supMapRack1, supMapRack2, supMapRack3, supMapRack4, supMapRack5);
+
+            //generate topologies
+            TopologyDetails topo1 = genTopology("topo-1", config, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
+            TopologyDetails topo2 = genTopology("topo-2", config, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
+
+            Topologies topologies = new Topologies(topo1, topo2);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+
+            List<String> supHostnames = new LinkedList<>();
+            for (SupervisorDetails sup : supMap.values()) {
+                supHostnames.add(sup.getHost());
+            }
+            Map<String, List<String>> rackToNodes = new HashMap<>();
+            Map<String, String> resolvedSuperVisors = TestNetworkTopographyPlugin.resolve(supHostnames);
+            for (Map.Entry<String, String> entry : resolvedSuperVisors.entrySet()) {
+                String hostName = entry.getKey();
+                String rack = entry.getValue();
+                rackToNodes.computeIfAbsent(rack, rid -> new ArrayList<>()).add(hostName);
+            }
+            cluster.setNetworkTopography(rackToNodes);
+
+            DefaultResourceAwareStrategyOld rs = new DefaultResourceAwareStrategyOld();
+
+            rs.prepareForScheduling(cluster, topo1);
+            INodeSorter nodeSorter = new NodeSorterHostProximity(cluster, topo1, BaseResourceAwareStrategy.NodeSortType.DEFAULT_RAS);
+            nodeSorter.prepare(null);
+            Iterable<ObjectResourcesItem> sortedRacks = nodeSorter.getSortedRacks();
+
+            Iterator<ObjectResourcesItem> it = sortedRacks.iterator();
+            // Ranked first since rack-0 has the most balanced set of resources
+            assertEquals("rack-0", it.next().id, "rack-0 should be ordered first");
+            // Ranked second since rack-1 has a balanced set of resources but less than rack-0
+            assertEquals("rack-1", it.next().id, "rack-1 should be ordered second");
+            // Ranked third since rack-4 has a lot of cpu but not a lot of memory
+            assertEquals("rack-4", it.next().id, "rack-4 should be ordered third");
+            // Ranked fourth since rack-3 has alot of memory but not cpu
+            assertEquals("rack-3", it.next().id, "rack-3 should be ordered fourth");
+            //Ranked fifth since rack-2 has not cpu resources
+            assertEquals("rack-2", it.next().id, "rack-2 should be ordered fifth");
+            //Ranked last since rack-5 has neither CPU nor memory available
+            assertEquals("rack-5", it.next().id, "Rack-5 should be ordered sixth");
+
+            SchedulingResult schedulingResult = rs.schedule(cluster, topo1);
+            assert (schedulingResult.isSuccess());
+            SchedulerAssignment assignment = cluster.getAssignmentById(topo1.getId());
+            for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
+                //make sure all workers on scheduled in rack-0
+                assertEquals("rack-0",
+                        resolvedSuperVisors.get(rs.idToNode(ws.getNodeId()).getHostname()), "assert worker scheduled on rack-0");
+            }
+            assertEquals(0, cluster.getUnassignedExecutors(topo1).size(), "All executors in topo-1 scheduled");
+
+            //Test if topology is already partially scheduled on one rack
+            Iterator<ExecutorDetails> executorIterator = topo2.getExecutors().iterator();
+            List<String> nodeHostnames = rackToNodes.get("rack-1");
+            for (int i = 0; i < topo2.getExecutors().size() / 2; i++) {
+                String nodeHostname = nodeHostnames.get(i % nodeHostnames.size());
+                RasNode node = rs.hostnameToNodes(nodeHostname).get(0);
+                WorkerSlot targetSlot = node.getFreeSlots().iterator().next();
+                ExecutorDetails targetExec = executorIterator.next();
+                // to keep track of free slots
+                node.assign(targetSlot, topo2, Collections.singletonList(targetExec));
+            }
 
-        rs = new DefaultResourceAwareStrategyOld();
-        // schedule topo2
-        schedulingResult = rs.schedule(cluster, topo2);
-        assert(schedulingResult.isSuccess());
-        assignment = cluster.getAssignmentById(topo2.getId());
-        for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
-            //make sure all workers on scheduled in rack-1
-            assertEquals("rack-1",
-                    resolvedSuperVisors.get(rs.idToNode(ws.getNodeId()).getHostname()), "assert worker scheduled on rack-1");
+            rs = new DefaultResourceAwareStrategyOld();
+            // schedule topo2
+            schedulingResult = rs.schedule(cluster, topo2);
+            assert (schedulingResult.isSuccess());
+            assignment = cluster.getAssignmentById(topo2.getId());
+            for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
+                //make sure all workers on scheduled in rack-1
+                assertEquals("rack-1",
+                        resolvedSuperVisors.get(rs.idToNode(ws.getNodeId()).getHostname()), "assert worker scheduled on rack-1");
+            }
+            assertEquals(0, cluster.getUnassignedExecutors(topo2).size(), "All executors in topo-2 scheduled");
         }
-        assertEquals(0, cluster.getUnassignedExecutors(topo2).size(), "All executors in topo-2 scheduled");
     }
 
     /**
@@ -845,126 +858,128 @@ public class TestDefaultResourceAwareStrategy {
      */
     @Test
     public void testMultipleRacksWithFavoritism() {
-        final Map<String, SupervisorDetails> supMap = new HashMap<>();
-        final Map<String, SupervisorDetails> supMapRack0 = genSupervisors(10, 4, 0, 400, 8000);
-        //generate another rack of supervisors with less resources
-        final Map<String, SupervisorDetails> supMapRack1 = genSupervisors(10, 4, 10, 200, 4000);
-
-        //generate some supervisors that are depleted of one resource
-        final Map<String, SupervisorDetails> supMapRack2 = genSupervisors(10, 4, 20, 0, 8000);
-
-        //generate some that has alot of memory but little of cpu
-        final Map<String, SupervisorDetails> supMapRack3 = genSupervisors(10, 4, 30, 10, 8000 * 2 + 4000);
-
-        //generate some that has alot of cpu but little of memory
-        final Map<String, SupervisorDetails> supMapRack4 = genSupervisors(10, 4, 40, 400 + 200 + 10, 1000);
-
-        supMap.putAll(supMapRack0);
-        supMap.putAll(supMapRack1);
-        supMap.putAll(supMapRack2);
-        supMap.putAll(supMapRack3);
-        supMap.putAll(supMapRack4);
-
-        Config config = createClusterConfig(100, 500, 500, null);
-        config.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE);
-        INimbus iNimbus = new INimbusTest();
-
-        //create test DNSToSwitchMapping plugin
-        DNSToSwitchMapping TestNetworkTopographyPlugin =
-            new TestDNSToSwitchMapping(supMapRack0, supMapRack1, supMapRack2, supMapRack3, supMapRack4);
-
-        Config t1Conf = new Config();
-        t1Conf.putAll(config);
-        final List<String> t1FavoredHostNames = Arrays.asList("host-41", "host-42", "host-43");
-        t1Conf.put(Config.TOPOLOGY_SCHEDULER_FAVORED_NODES, t1FavoredHostNames);
-        final List<String> t1UnfavoredHostIds = Arrays.asList("host-1", "host-2", "host-3");
-        t1Conf.put(Config.TOPOLOGY_SCHEDULER_UNFAVORED_NODES, t1UnfavoredHostIds);
-        //generate topologies
-        TopologyDetails topo1 = genTopology("topo-1", t1Conf, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
-
-
-        Config t2Conf = new Config();
-        t2Conf.putAll(config);
-        t2Conf.put(Config.TOPOLOGY_SCHEDULER_FAVORED_NODES, Arrays.asList("host-31", "host-32", "host-33"));
-        t2Conf.put(Config.TOPOLOGY_SCHEDULER_UNFAVORED_NODES, Arrays.asList("host-11", "host-12", "host-13"));
-        TopologyDetails topo2 = genTopology("topo-2", t2Conf, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
-
-        Topologies topologies = new Topologies(topo1, topo2);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-
-        List<String> supHostnames = new LinkedList<>();
-        for (SupervisorDetails sup : supMap.values()) {
-            supHostnames.add(sup.getHost());
-        }
-        Map<String, List<String>> rackToNodes = new HashMap<>();
-        Map<String, String> resolvedSuperVisors =  TestNetworkTopographyPlugin.resolve(supHostnames);
-        for (Map.Entry<String, String> entry : resolvedSuperVisors.entrySet()) {
-            String hostName = entry.getKey();
-            String rack = entry.getValue();
-            List<String> nodesForRack = rackToNodes.get(rack);
-            if (nodesForRack == null) {
-                nodesForRack = new ArrayList<>();
-                rackToNodes.put(rack, nodesForRack);
+        for (Class strategyClass : strategyClasses) {
+            final Map<String, SupervisorDetails> supMap = new HashMap<>();
+            final Map<String, SupervisorDetails> supMapRack0 = genSupervisors(10, 4, 0, 400, 8000);
+            //generate another rack of supervisors with less resources
+            final Map<String, SupervisorDetails> supMapRack1 = genSupervisors(10, 4, 10, 200, 4000);
+
+            //generate some supervisors that are depleted of one resource
+            final Map<String, SupervisorDetails> supMapRack2 = genSupervisors(10, 4, 20, 0, 8000);
+
+            //generate some that has alot of memory but little of cpu
+            final Map<String, SupervisorDetails> supMapRack3 = genSupervisors(10, 4, 30, 10, 8000 * 2 + 4000);
+
+            //generate some that has alot of cpu but little of memory
+            final Map<String, SupervisorDetails> supMapRack4 = genSupervisors(10, 4, 40, 400 + 200 + 10, 1000);
+
+            supMap.putAll(supMapRack0);
+            supMap.putAll(supMapRack1);
+            supMap.putAll(supMapRack2);
+            supMap.putAll(supMapRack3);
+            supMap.putAll(supMapRack4);
+
+            Config config = createClusterConfig(strategyClass, 100, 500, 500, null);
+            config.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE);
+            INimbus iNimbus = new INimbusTest();
+
+            //create test DNSToSwitchMapping plugin
+            DNSToSwitchMapping TestNetworkTopographyPlugin =
+                    new TestDNSToSwitchMapping(supMapRack0, supMapRack1, supMapRack2, supMapRack3, supMapRack4);
+
+            Config t1Conf = new Config();
+            t1Conf.putAll(config);
+            final List<String> t1FavoredHostNames = Arrays.asList("host-41", "host-42", "host-43");
+            t1Conf.put(Config.TOPOLOGY_SCHEDULER_FAVORED_NODES, t1FavoredHostNames);
+            final List<String> t1UnfavoredHostIds = Arrays.asList("host-1", "host-2", "host-3");
+            t1Conf.put(Config.TOPOLOGY_SCHEDULER_UNFAVORED_NODES, t1UnfavoredHostIds);
+            //generate topologies
+            TopologyDetails topo1 = genTopology("topo-1", t1Conf, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
+
+
+            Config t2Conf = new Config();
+            t2Conf.putAll(config);
+            t2Conf.put(Config.TOPOLOGY_SCHEDULER_FAVORED_NODES, Arrays.asList("host-31", "host-32", "host-33"));
+            t2Conf.put(Config.TOPOLOGY_SCHEDULER_UNFAVORED_NODES, Arrays.asList("host-11", "host-12", "host-13"));
+            TopologyDetails topo2 = genTopology("topo-2", t2Conf, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
+
+            Topologies topologies = new Topologies(topo1, topo2);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+
+            List<String> supHostnames = new LinkedList<>();
+            for (SupervisorDetails sup : supMap.values()) {
+                supHostnames.add(sup.getHost());
+            }
+            Map<String, List<String>> rackToNodes = new HashMap<>();
+            Map<String, String> resolvedSuperVisors = TestNetworkTopographyPlugin.resolve(supHostnames);
+            for (Map.Entry<String, String> entry : resolvedSuperVisors.entrySet()) {
+                String hostName = entry.getKey();
+                String rack = entry.getValue();
+                List<String> nodesForRack = rackToNodes.get(rack);
+                if (nodesForRack == null) {
+                    nodesForRack = new ArrayList<>();
+                    rackToNodes.put(rack, nodesForRack);
+                }
+                nodesForRack.add(hostName);
+            }
+            cluster.setNetworkTopography(rackToNodes);
+
+            DefaultResourceAwareStrategyOld rs = new DefaultResourceAwareStrategyOld();
+
+            rs.prepareForScheduling(cluster, topo1);
+            INodeSorter nodeSorter = new NodeSorterHostProximity(cluster, topo1, BaseResourceAwareStrategy.NodeSortType.DEFAULT_RAS);
+            nodeSorter.prepare(null);
+            Iterable<ObjectResourcesItem> sortedRacks = nodeSorter.getSortedRacks();
+
+            Iterator<ObjectResourcesItem> it = sortedRacks.iterator();
+            // Ranked first since rack-0 has the most balanced set of resources
+            assertEquals("rack-0", it.next().id, "rack-0 should be ordered first");
+            // Ranked second since rack-1 has a balanced set of resources but less than rack-0
+            assertEquals("rack-1", it.next().id, "rack-1 should be ordered second");
+            // Ranked third since rack-4 has a lot of cpu but not a lot of memory
+            assertEquals("rack-4", it.next().id, "rack-4 should be ordered third");
+            // Ranked fourth since rack-3 has alot of memory but not cpu
+            assertEquals("rack-3", it.next().id, "rack-3 should be ordered fourth");
+            //Ranked last since rack-2 has not cpu resources
+            assertEquals("rack-2", it.next().id, "rack-2 should be ordered fifth");
+
+            SchedulingResult schedulingResult = rs.schedule(cluster, topo1);
+            assert (schedulingResult.isSuccess());
+            SchedulerAssignment assignment = cluster.getAssignmentById(topo1.getId());
+            for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
+                String hostName = rs.idToNode(ws.getNodeId()).getHostname();
+                String rackId = resolvedSuperVisors.get(hostName);
+                assertTrue(t1FavoredHostNames.contains(hostName) || "rack-0".equals(rackId),
+                        ws + " is neither on a favored node " + t1FavoredHostNames + " nor the highest priority rack (rack-0)");
+                assertFalse(t1UnfavoredHostIds.contains(hostName),
+                        ws + " is a part of an unfavored node " + t1UnfavoredHostIds);
+            }
+            assertEquals(0, cluster.getUnassignedExecutors(topo1).size(), "All executors in topo-1 scheduled");
+
+            //Test if topology is already partially scheduled on one rack
+            Iterator<ExecutorDetails> executorIterator = topo2.getExecutors().iterator();
+            List<String> nodeHostnames = rackToNodes.get("rack-1");
+            for (int i = 0; i < topo2.getExecutors().size() / 2; i++) {
+                String nodeHostname = nodeHostnames.get(i % nodeHostnames.size());
+                RasNode node = rs.hostnameToNodes(nodeHostname).get(0);
+                WorkerSlot targetSlot = node.getFreeSlots().iterator().next();
+                ExecutorDetails targetExec = executorIterator.next();
+                // to keep track of free slots
+                node.assign(targetSlot, topo2, Collections.singletonList(targetExec));
             }
-            nodesForRack.add(hostName);
-        }
-        cluster.setNetworkTopography(rackToNodes);
-
-        DefaultResourceAwareStrategyOld rs = new DefaultResourceAwareStrategyOld();
-
-        rs.prepareForScheduling(cluster, topo1);
-        INodeSorter nodeSorter = new NodeSorterHostProximity(cluster, topo1, BaseResourceAwareStrategy.NodeSortType.DEFAULT_RAS);
-        nodeSorter.prepare(null);
-        Iterable<ObjectResourcesItem> sortedRacks= nodeSorter.getSortedRacks();
-
-        Iterator<ObjectResourcesItem> it = sortedRacks.iterator();
-        // Ranked first since rack-0 has the most balanced set of resources
-        assertEquals("rack-0", it.next().id, "rack-0 should be ordered first");
-        // Ranked second since rack-1 has a balanced set of resources but less than rack-0
-        assertEquals("rack-1", it.next().id, "rack-1 should be ordered second");
-        // Ranked third since rack-4 has a lot of cpu but not a lot of memory
-        assertEquals("rack-4", it.next().id, "rack-4 should be ordered third");
-        // Ranked fourth since rack-3 has alot of memory but not cpu
-        assertEquals("rack-3", it.next().id, "rack-3 should be ordered fourth");
-        //Ranked last since rack-2 has not cpu resources
-        assertEquals("rack-2", it.next().id, "rack-2 should be ordered fifth");
-
-        SchedulingResult schedulingResult = rs.schedule(cluster, topo1);
-        assert(schedulingResult.isSuccess());
-        SchedulerAssignment assignment = cluster.getAssignmentById(topo1.getId());
-        for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
-            String hostName = rs.idToNode(ws.getNodeId()).getHostname();
-            String rackId = resolvedSuperVisors.get(hostName);
-            assertTrue(t1FavoredHostNames.contains(hostName) || "rack-0".equals(rackId),
-                ws + " is neither on a favored node " + t1FavoredHostNames + " nor the highest priority rack (rack-0)");
-            assertFalse(t1UnfavoredHostIds.contains(hostName),
-                ws + " is a part of an unfavored node " + t1UnfavoredHostIds);
-        }
-        assertEquals(0, cluster.getUnassignedExecutors(topo1).size(), "All executors in topo-1 scheduled");
-
-        //Test if topology is already partially scheduled on one rack
-        Iterator<ExecutorDetails> executorIterator = topo2.getExecutors().iterator();
-        List<String> nodeHostnames = rackToNodes.get("rack-1");
-        for (int i = 0; i< topo2.getExecutors().size()/2; i++) {
-            String nodeHostname = nodeHostnames.get(i % nodeHostnames.size());
-            RasNode node = rs.hostnameToNodes(nodeHostname).get(0);
-            WorkerSlot targetSlot = node.getFreeSlots().iterator().next();
-            ExecutorDetails targetExec = executorIterator.next();
-            // to keep track of free slots
-            node.assign(targetSlot, topo2, Collections.singletonList(targetExec));
-        }
 
-        rs = new DefaultResourceAwareStrategyOld();
-        // schedule topo2
-        schedulingResult = rs.schedule(cluster, topo2);
-        assert(schedulingResult.isSuccess());
-        assignment = cluster.getAssignmentById(topo2.getId());
-        for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
-            //make sure all workers on scheduled in rack-1
-            // The favored nodes would have put it on a different rack, but because that rack does not have free space to run the
-            // topology it falls back to this rack
-            assertEquals("rack-1", resolvedSuperVisors.get(rs.idToNode(ws.getNodeId()).getHostname()), "assert worker scheduled on rack-1");
+            rs = new DefaultResourceAwareStrategyOld();
+            // schedule topo2
+            schedulingResult = rs.schedule(cluster, topo2);
+            assert (schedulingResult.isSuccess());
+            assignment = cluster.getAssignmentById(topo2.getId());
+            for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
+                //make sure all workers on scheduled in rack-1
+                // The favored nodes would have put it on a different rack, but because that rack does not have free space to run the
+                // topology it falls back to this rack
+                assertEquals("rack-1", resolvedSuperVisors.get(rs.idToNode(ws.getNodeId()).getHostname()), "assert worker scheduled on rack-1");
+            }
+            assertEquals(0, cluster.getUnassignedExecutors(topo2).size(), "All executors in topo-2 scheduled");
         }
-        assertEquals(0, cluster.getUnassignedExecutors(topo2).size() , "All executors in topo-2 scheduled" );
     }
 }
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestGenericResourceAwareStrategy.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestGenericResourceAwareStrategy.java
index 26ea6b4a4..1bf694041 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestGenericResourceAwareStrategy.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestGenericResourceAwareStrategy.java
@@ -67,6 +67,10 @@ import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
 
 public class TestGenericResourceAwareStrategy {
     private static final Logger LOG = LoggerFactory.getLogger(TestGenericResourceAwareStrategy.class);
+    private static final Class[] strategyClasses = {
+            GenericResourceAwareStrategy.class,
+            GenericResourceAwareStrategyOld.class,
+    };
 
     private final int currentTime = 1450418597;
     private IScheduler scheduler = null;
@@ -79,14 +83,10 @@ public class TestGenericResourceAwareStrategy {
         }
     }
 
-    protected Class getGenericResourceAwareStrategyClass() {
-        return GenericResourceAwareStrategy.class;
-    }
-
-    private Config createGrasClusterConfig(double compPcore, double compOnHeap, double compOffHeap,
+    private Config createGrasClusterConfig(Class strategyClass, double compPcore, double compOnHeap, double compOffHeap,
                                                  Map<String, Map<String, Number>> pools, Map<String, Double> genericResourceMap) {
         Config config = TestUtilsForResourceAwareScheduler.createGrasClusterConfig(compPcore, compOnHeap, compOffHeap, pools, genericResourceMap);
-        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, getGenericResourceAwareStrategyClass().getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, strategyClass.getName());
         return config;
     }
 
@@ -95,93 +95,95 @@ public class TestGenericResourceAwareStrategy {
      */
     @Test
     public void testGenericResourceAwareStrategySharedMemory() {
-        int spoutParallelism = 2;
-        int boltParallelism = 2;
-        int numBolts = 3;
-        double cpuPercent = 10;
-        double memoryOnHeap = 10;
-        double memoryOffHeap = 10;
-        double sharedOnHeap = 500;
-        double sharedOffHeapNode = 700;
-        double sharedOffHeapWorker = 500;
-        TopologyBuilder builder = new TopologyBuilder();
-        builder.setSpout("spout", new TestSpout(),
-                spoutParallelism).addResource("gpu.count", 1.0);
-        builder.setBolt("bolt-1", new TestBolt(),
-                boltParallelism).addSharedMemory(new SharedOffHeapWithinWorker(sharedOffHeapWorker, "bolt-1 shared off heap worker")).shuffleGrouping("spout");
-        builder.setBolt("bolt-2", new TestBolt(),
-                boltParallelism).addSharedMemory(new SharedOffHeapWithinNode(sharedOffHeapNode, "bolt-2 shared node")).shuffleGrouping("bolt-1");
-        builder.setBolt("bolt-3", new TestBolt(),
-                boltParallelism).addSharedMemory(new SharedOnHeap(sharedOnHeap, "bolt-3 shared worker")).shuffleGrouping("bolt-2");
-
-        StormTopology stormTopology = builder.createTopology();
-
-        INimbus iNimbus = new INimbusTest();
-
-        Config conf = createGrasClusterConfig(cpuPercent, memoryOnHeap, memoryOffHeap, null, Collections.emptyMap());
-        Map<String, Double> genericResourcesMap = new HashMap<>();
-        genericResourcesMap.put("gpu.count", 1.0);
-
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 500, 2000, genericResourcesMap);
-
-        conf.put(Config.TOPOLOGY_PRIORITY, 0);
-        conf.put(Config.TOPOLOGY_NAME, "testTopology");
-        conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
-        TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormTopology, 0,
-                genExecsAndComps(stormTopology), currentTime, "user");
-
-        Topologies topologies = new Topologies(topo);
-
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
-
-        scheduler = new ResourceAwareScheduler();
+        for (Class strategyClass: strategyClasses) {
+            int spoutParallelism = 2;
+            int boltParallelism = 2;
+            int numBolts = 3;
+            double cpuPercent = 10;
+            double memoryOnHeap = 10;
+            double memoryOffHeap = 10;
+            double sharedOnHeap = 500;
+            double sharedOffHeapNode = 700;
+            double sharedOffHeapWorker = 500;
+            TopologyBuilder builder = new TopologyBuilder();
+            builder.setSpout("spout", new TestSpout(),
+                    spoutParallelism).addResource("gpu.count", 1.0);
+            builder.setBolt("bolt-1", new TestBolt(),
+                    boltParallelism).addSharedMemory(new SharedOffHeapWithinWorker(sharedOffHeapWorker, "bolt-1 shared off heap worker")).shuffleGrouping("spout");
+            builder.setBolt("bolt-2", new TestBolt(),
+                    boltParallelism).addSharedMemory(new SharedOffHeapWithinNode(sharedOffHeapNode, "bolt-2 shared node")).shuffleGrouping("bolt-1");
+            builder.setBolt("bolt-3", new TestBolt(),
+                    boltParallelism).addSharedMemory(new SharedOnHeap(sharedOnHeap, "bolt-3 shared worker")).shuffleGrouping("bolt-2");
+
+            StormTopology stormTopology = builder.createTopology();
+
+            INimbus iNimbus = new INimbusTest();
+
+            Config conf = createGrasClusterConfig(strategyClass, cpuPercent, memoryOnHeap, memoryOffHeap, null, Collections.emptyMap());
+            Map<String, Double> genericResourcesMap = new HashMap<>();
+            genericResourcesMap.put("gpu.count", 1.0);
+
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 500, 2000, genericResourcesMap);
+
+            conf.put(Config.TOPOLOGY_PRIORITY, 0);
+            conf.put(Config.TOPOLOGY_NAME, "testTopology");
+            conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
+            TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormTopology, 0,
+                    genExecsAndComps(stormTopology), currentTime, "user");
+
+            Topologies topologies = new Topologies(topo);
+
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
+
+            scheduler = new ResourceAwareScheduler();
+
+            scheduler.prepare(conf, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+
+            for (Entry<String, SupervisorResources> entry : cluster.getSupervisorsResourcesMap().entrySet()) {
+                String supervisorId = entry.getKey();
+                SupervisorResources resources = entry.getValue();
+                assertTrue(resources.getTotalCpu() >= resources.getUsedCpu(), supervisorId);
+                assertTrue(resources.getTotalMem() >= resources.getUsedMem(), supervisorId);
+            }
 
-        scheduler.prepare(conf, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-        
-        for (Entry<String, SupervisorResources> entry: cluster.getSupervisorsResourcesMap().entrySet()) {
-            String supervisorId = entry.getKey();
-            SupervisorResources resources = entry.getValue();
-            assertTrue(resources.getTotalCpu() >= resources.getUsedCpu(), supervisorId);
-            assertTrue(resources.getTotalMem() >= resources.getUsedMem(), supervisorId);
-        }
+            // If we didn't take GPUs into account everything would fit under a single slot
+            // But because there is only 1 GPU per node, and each of the 2 spouts needs a GPU
+            // It has to be scheduled on at least 2 nodes, and hence 2 slots.
+            // Because of this, all the bolts will be scheduled on a single slot with one of
+            // the spouts and the other spout is on its own slot.  So everything that can be shared is
+            // shared.
+            int totalNumberOfTasks = (spoutParallelism + (boltParallelism * numBolts));
+            double totalExpectedCPU = totalNumberOfTasks * cpuPercent;
+            double totalExpectedOnHeap = (totalNumberOfTasks * memoryOnHeap) + sharedOnHeap;
+            double totalExpectedWorkerOffHeap = (totalNumberOfTasks * memoryOffHeap) + sharedOffHeapWorker;
+
+            SchedulerAssignment assignment = cluster.getAssignmentById(topo.getId());
+            Set<WorkerSlot> slots = assignment.getSlots();
+            Map<String, Double> nodeToTotalShared = assignment.getNodeIdToTotalSharedOffHeapNodeMemory();
+            LOG.info("NODE TO SHARED OFF HEAP {}", nodeToTotalShared);
+            Map<WorkerSlot, WorkerResources> scheduledResources = assignment.getScheduledResources();
+            assertEquals(2, slots.size());
+            assertEquals(2, nodeToTotalShared.size());
+            assertEquals(2, scheduledResources.size());
+            double totalFoundCPU = 0.0;
+            double totalFoundOnHeap = 0.0;
+            double totalFoundWorkerOffHeap = 0.0;
+            for (WorkerSlot ws : slots) {
+                WorkerResources resources = scheduledResources.get(ws);
+                totalFoundCPU += resources.get_cpu();
+                totalFoundOnHeap += resources.get_mem_on_heap();
+                totalFoundWorkerOffHeap += resources.get_mem_off_heap();
+            }
 
-        // If we didn't take GPUs into account everything would fit under a single slot
-        // But because there is only 1 GPU per node, and each of the 2 spouts needs a GPU
-        // It has to be scheduled on at least 2 nodes, and hence 2 slots.
-        // Because of this, all the bolts will be scheduled on a single slot with one of
-        // the spouts and the other spout is on its own slot.  So everything that can be shared is
-        // shared.
-        int totalNumberOfTasks = (spoutParallelism + (boltParallelism * numBolts));
-        double totalExpectedCPU = totalNumberOfTasks * cpuPercent;
-        double totalExpectedOnHeap = (totalNumberOfTasks * memoryOnHeap) + sharedOnHeap;
-        double totalExpectedWorkerOffHeap = (totalNumberOfTasks * memoryOffHeap) + sharedOffHeapWorker;
-        
-        SchedulerAssignment assignment = cluster.getAssignmentById(topo.getId());
-        Set<WorkerSlot> slots = assignment.getSlots();
-        Map<String, Double> nodeToTotalShared = assignment.getNodeIdToTotalSharedOffHeapNodeMemory();
-        LOG.info("NODE TO SHARED OFF HEAP {}", nodeToTotalShared);
-        Map<WorkerSlot, WorkerResources> scheduledResources = assignment.getScheduledResources();
-        assertEquals(2, slots.size());
-        assertEquals(2, nodeToTotalShared.size());
-        assertEquals(2, scheduledResources.size());
-        double totalFoundCPU = 0.0;
-        double totalFoundOnHeap = 0.0;
-        double totalFoundWorkerOffHeap = 0.0;
-        for (WorkerSlot ws : slots) {
-            WorkerResources resources = scheduledResources.get(ws);
-            totalFoundCPU += resources.get_cpu();
-            totalFoundOnHeap += resources.get_mem_on_heap();
-            totalFoundWorkerOffHeap += resources.get_mem_off_heap();
+            assertEquals(totalExpectedCPU, totalFoundCPU, 0.01);
+            assertEquals(totalExpectedOnHeap, totalFoundOnHeap, 0.01);
+            assertEquals(totalExpectedWorkerOffHeap, totalFoundWorkerOffHeap, 0.01);
+            assertEquals(sharedOffHeapNode, nodeToTotalShared.values().stream().mapToDouble((d) -> d).sum(), 0.01);
+            assertEquals(sharedOnHeap, scheduledResources.values().stream().mapToDouble(WorkerResources::get_shared_mem_on_heap).sum(), 0.01);
+            assertEquals(sharedOffHeapWorker, scheduledResources.values().stream().mapToDouble(WorkerResources::get_shared_mem_off_heap).sum(),
+                    0.01);
         }
-
-        assertEquals(totalExpectedCPU, totalFoundCPU, 0.01);
-        assertEquals(totalExpectedOnHeap, totalFoundOnHeap, 0.01);
-        assertEquals(totalExpectedWorkerOffHeap, totalFoundWorkerOffHeap, 0.01);
-        assertEquals(sharedOffHeapNode, nodeToTotalShared.values().stream().mapToDouble((d) -> d).sum(), 0.01);
-        assertEquals(sharedOnHeap, scheduledResources.values().stream().mapToDouble(WorkerResources::get_shared_mem_on_heap).sum(), 0.01);
-        assertEquals(sharedOffHeapWorker, scheduledResources.values().stream().mapToDouble(WorkerResources::get_shared_mem_off_heap).sum(),
-            0.01);
     }
 
     /**
@@ -194,228 +196,232 @@ public class TestGenericResourceAwareStrategy {
     @ValueSource(ints = {-1, 0, 1, 2})
     public void testGenericResourceAwareStrategyWithoutSettingAckerExecutors(int numOfAckersPerWorker)
         throws InvalidTopologyException {
-        int spoutParallelism = 1;
-        int boltParallelism = 2;
-        TopologyBuilder builder = new TopologyBuilder();
-        builder.setSpout("spout", new TestSpout(),
-                spoutParallelism);
-        builder.setBolt("bolt-1", new TestBolt(),
-                boltParallelism).shuffleGrouping("spout");
-        builder.setBolt("bolt-2", new TestBolt(),
-                boltParallelism).shuffleGrouping("bolt-1").addResource("gpu.count", 1.0);
-        builder.setBolt("bolt-3", new TestBolt(),
-                boltParallelism).shuffleGrouping("bolt-2").addResource("gpu.count", 2.0);
-
-        String topoName = "testTopology";
-        StormTopology stormTopology = builder.createTopology();
-
-        INimbus iNimbus = new INimbusTest();
-
-        Config conf = createGrasClusterConfig(50, 500, 0, null, Collections.emptyMap());
-        Map<String, Double> genericResourcesMap = new HashMap<>();
-        genericResourcesMap.put("gpu.count", 2.0);
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 200, 2000, genericResourcesMap);
-
-
-        conf.put(Config.TOPOLOGY_PRIORITY, 0);
-        conf.put(Config.TOPOLOGY_NAME, topoName);
-        conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
-        conf.put(Config.TOPOLOGY_SUBMITTER_USER, "user");
-
-        // Topology needs 2 workers (estimated by nimbus based on resources),
-        // but with ackers added, probably more worker will be launched.
-        // Parameterized test on different numOfAckersPerWorker
-        if (numOfAckersPerWorker == -1) {
-            // Both Config.TOPOLOGY_ACKER_EXECUTORS and Config.TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER are not set
-            // Default will be 2 (estimate num of workers) and 1 respectively
-        } else {
-            conf.put(Config.TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER, numOfAckersPerWorker);
+        for (Class strategyClass: strategyClasses) {
+            int spoutParallelism = 1;
+            int boltParallelism = 2;
+            TopologyBuilder builder = new TopologyBuilder();
+            builder.setSpout("spout", new TestSpout(),
+                    spoutParallelism);
+            builder.setBolt("bolt-1", new TestBolt(),
+                    boltParallelism).shuffleGrouping("spout");
+            builder.setBolt("bolt-2", new TestBolt(),
+                    boltParallelism).shuffleGrouping("bolt-1").addResource("gpu.count", 1.0);
+            builder.setBolt("bolt-3", new TestBolt(),
+                    boltParallelism).shuffleGrouping("bolt-2").addResource("gpu.count", 2.0);
+
+            String topoName = "testTopology";
+            StormTopology stormTopology = builder.createTopology();
+
+            INimbus iNimbus = new INimbusTest();
+
+            Config conf = createGrasClusterConfig(strategyClass, 50, 500, 0, null, Collections.emptyMap());
+            Map<String, Double> genericResourcesMap = new HashMap<>();
+            genericResourcesMap.put("gpu.count", 2.0);
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 200, 2000, genericResourcesMap);
+
+
+            conf.put(Config.TOPOLOGY_PRIORITY, 0);
+            conf.put(Config.TOPOLOGY_NAME, topoName);
+            conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
+            conf.put(Config.TOPOLOGY_SUBMITTER_USER, "user");
+
+            // Topology needs 2 workers (estimated by nimbus based on resources),
+            // but with ackers added, probably more worker will be launched.
+            // Parameterized test on different numOfAckersPerWorker
+            if (numOfAckersPerWorker == -1) {
+                // Both Config.TOPOLOGY_ACKER_EXECUTORS and Config.TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER are not set
+                // Default will be 2 (estimate num of workers) and 1 respectively
+            } else {
+                conf.put(Config.TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER, numOfAckersPerWorker);
+            }
+
+            int estimatedNumWorker = ServerUtils.getEstimatedWorkerCountForRasTopo(conf, stormTopology);
+            Nimbus.setUpAckerExecutorConfigs(topoName, conf, conf, estimatedNumWorker);
+
+            conf.put(Config.TOPOLOGY_ACKER_RESOURCES_ONHEAP_MEMORY_MB, 250);
+            conf.put(Config.TOPOLOGY_ACKER_CPU_PCORE_PERCENT, 50);
+
+            TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormTopology, 0,
+                    genExecsAndComps(StormCommon.systemTopology(conf, stormTopology)), currentTime, "user");
+
+            Topologies topologies = new Topologies(topo);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
+
+            scheduler = new ResourceAwareScheduler();
+
+            scheduler.prepare(conf, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+
+            // We need to have 3 slots on 3 separate hosts. The topology needs 6 GPUs 3500 MB memory and 350% CPU
+            // The bolt-3 instances must be on separate nodes because they each need 2 GPUs.
+            // The bolt-2 instances must be on the same node as they each need 1 GPU
+            // (this assumes that we are packing the components to avoid fragmentation).
+            // The bolt-1 and spout instances fill in the rest.
+
+            // Ordered execs: [[6, 6], [2, 2], [4, 4], [5, 5], [1, 1], [3, 3], [0, 0]]
+            // Ackers: [[8, 8], [7, 7]] (+ [[9, 9], [10, 10]] when numOfAckersPerWorker=2)
+            HashSet<HashSet<ExecutorDetails>> expectedScheduling = new HashSet<>();
+            if (numOfAckersPerWorker == -1 || numOfAckersPerWorker == 1) {
+                expectedScheduling.add(new HashSet<>(Collections.singletonList(
+                        new ExecutorDetails(3, 3)))); //bolt-3 - 500 MB, 50% CPU, 2 GPU
+                //Total 500 MB, 50% CPU, 2 - GPU -> this node has 1500 MB, 150% cpu, 0 GPU left
+                expectedScheduling.add(new HashSet<>(Arrays.asList(
+                        new ExecutorDetails(6, 6), //bolt-2 - 500 MB, 50% CPU, 1 GPU
+                        new ExecutorDetails(2, 2), //bolt-1 - 500 MB, 50% CPU, 0 GPU
+                        new ExecutorDetails(5, 5), //bolt-2 - 500 MB, 50% CPU, 1 GPU
+                        new ExecutorDetails(8, 8)))); //acker - 250 MB, 50% CPU, 0 GPU
+                //Total 1750 MB, 200% CPU, 2 GPU -> this node has 250 MB, 0% CPU, 0 GPU left
+                expectedScheduling.add(new HashSet<>(Arrays.asList(
+                        new ExecutorDetails(4, 4), //bolt-3 500 MB, 50% cpu, 2 GPU
+                        new ExecutorDetails(1, 1), //bolt-1 - 500 MB, 50% CPU, 0 GPU
+                        new ExecutorDetails(0, 0), //Spout - 500 MB, 50% CPU, 0 GPU
+                        new ExecutorDetails(7, 7)))); //acker - 250 MB, 50% CPU, 0 GPU
+                //Total 1750 MB, 200% CPU, 2 GPU -> this node has 250 MB, 0% CPU, 0 GPU left
+            } else if (numOfAckersPerWorker == 0) {
+                expectedScheduling.add(new HashSet<>(Collections.singletonList(
+                        new ExecutorDetails(3, 3)))); //bolt-3 - 500 MB, 50% CPU, 2 GPU
+                //Total 500 MB, 50% CPU, 2 - GPU -> this node has 1500 MB, 150% cpu, 0 GPU left
+                expectedScheduling.add(new HashSet<>(Arrays.asList(
+                        new ExecutorDetails(6, 6), //bolt-2 - 500 MB, 50% CPU, 1 GPU
+                        new ExecutorDetails(2, 2), //bolt-1 - 500 MB, 50% CPU, 0 GPU
+                        new ExecutorDetails(5, 5), //bolt-2 - 500 MB, 50% CPU, 1 GPU
+                        new ExecutorDetails(1, 1))));  //bolt-1 - 500 MB, 50% CPU, 0 GPU
+                //Total 2000 MB, 200% CPU, 2 GPU -> this node has 0 MB, 0% CPU, 0 GPU left
+                expectedScheduling.add(new HashSet<>(Arrays.asList(
+                        new ExecutorDetails(0, 0), //Spout - 500 MB, 50% CPU, 0 GPU
+                        new ExecutorDetails(4, 4)))); //bolt-3 500 MB, 50% cpu, 2 GPU
+                //Total 1000 MB, 100% CPU, 2 GPU -> this node has 1000 MB, 100% CPU, 0 GPU left
+            } else if (numOfAckersPerWorker == 2) {
+                expectedScheduling.add(new HashSet<>(Collections.singletonList(
+                        new ExecutorDetails(3, 3)))); //bolt-3 - 500 MB, 50% CPU, 2 GPU
+                //Total 500 MB, 50% CPU, 2 - GPU -> this node has 1500 MB, 150% cpu, 0 GPU left
+                expectedScheduling.add(new HashSet<>(Arrays.asList(
+                        new ExecutorDetails(7, 7),      //acker - 250 MB, 50% CPU, 0 GPU
+                        new ExecutorDetails(8, 8),      //acker - 250 MB, 50% CPU, 0 GPU
+                        new ExecutorDetails(6, 6),      //bolt-2 - 500 MB, 50% CPU, 1 GPU
+                        new ExecutorDetails(2, 2))));   //bolt-1 - 500 MB, 50% CPU, 0 GPU
+                //Total 1500 MB, 200% CPU, 2 GPU -> this node has 500 MB, 0% CPU, 0 GPU left
+                expectedScheduling.add(new HashSet<>(Arrays.asList(
+                        new ExecutorDetails(9, 9),    //acker- 250 MB, 50% CPU, 0 GPU
+                        new ExecutorDetails(10, 10),  //acker- 250 MB, 50% CPU, 0 GPU
+                        new ExecutorDetails(1, 1),    //bolt-1 - 500 MB, 50% CPU, 0 GPU
+                        new ExecutorDetails(4, 4)))); //bolt-3 500 MB, 50% cpu, 2 GPU
+                //Total 1500 MB, 200% CPU, 2 GPU -> this node has 500 MB, 0% CPU, 0 GPU left
+                expectedScheduling.add(new HashSet<>(Arrays.asList(
+                        new ExecutorDetails(0, 0), //Spout - 500 MB, 50% CPU, 0 GPU
+                        new ExecutorDetails(5, 5)))); //bolt-2 - 500 MB, 50% CPU, 1 GPU
+                //Total 1000 MB, 100% CPU, 2 GPU -> this node has 1000 MB, 100% CPU, 0 GPU left
+            }
+            HashSet<HashSet<ExecutorDetails>> foundScheduling = new HashSet<>();
+            SchedulerAssignment assignment = cluster.getAssignmentById("testTopology-id");
+            for (Collection<ExecutorDetails> execs : assignment.getSlotToExecutors().values()) {
+                foundScheduling.add(new HashSet<>(execs));
+            }
+
+            assertEquals(expectedScheduling, foundScheduling);
         }
+    }
+
+    /**
+     * Test if the scheduling logic for the GenericResourceAwareStrategy is correct
+     * with setting {@link Config#TOPOLOGY_ACKER_EXECUTORS}.
+     *
+     * Test details refer to {@link TestDefaultResourceAwareStrategy#testDefaultResourceAwareStrategyWithSettingAckerExecutors(int)}
+     */
+    @ParameterizedTest
+    @ValueSource(ints = {-1, 0, 2, 200})
+    public void testGenericResourceAwareStrategyWithSettingAckerExecutors(int numOfAckersPerWorker)
+        throws InvalidTopologyException {
+        for (Class strategyClass: strategyClasses) {
+            int spoutParallelism = 1;
+            int boltParallelism = 2;
+            TopologyBuilder builder = new TopologyBuilder();
+            builder.setSpout("spout", new TestSpout(),
+                    spoutParallelism);
+            builder.setBolt("bolt-1", new TestBolt(),
+                    boltParallelism).shuffleGrouping("spout");
+            builder.setBolt("bolt-2", new TestBolt(),
+                    boltParallelism).shuffleGrouping("bolt-1").addResource("gpu.count", 1.0);
+            builder.setBolt("bolt-3", new TestBolt(),
+                    boltParallelism).shuffleGrouping("bolt-2").addResource("gpu.count", 2.0);
+
+            String topoName = "testTopology";
+            StormTopology stormTopology = builder.createTopology();
+
+            INimbus iNimbus = new INimbusTest();
+
+            Config conf = createGrasClusterConfig(strategyClass, 50, 500, 0, null, Collections.emptyMap());
+            Map<String, Double> genericResourcesMap = new HashMap<>();
+            genericResourcesMap.put("gpu.count", 2.0);
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 200, 2000, genericResourcesMap);
+
+
+            conf.put(Config.TOPOLOGY_PRIORITY, 0);
+            conf.put(Config.TOPOLOGY_NAME, topoName);
+            conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
+            conf.put(Config.TOPOLOGY_SUBMITTER_USER, "user");
+
+            conf.put(Config.TOPOLOGY_ACKER_EXECUTORS, 4);
+            if (numOfAckersPerWorker == -1) {
+                // Leave topology.acker.executors.per.worker unset
+            } else {
+                conf.put(Config.TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER, numOfAckersPerWorker);
+            }
 
-        int estimatedNumWorker = ServerUtils.getEstimatedWorkerCountForRasTopo(conf, stormTopology);
-        Nimbus.setUpAckerExecutorConfigs(topoName, conf, conf, estimatedNumWorker);
+            int estimatedNumWorker = ServerUtils.getEstimatedWorkerCountForRasTopo(conf, stormTopology);
+            Nimbus.setUpAckerExecutorConfigs(topoName, conf, conf, estimatedNumWorker);
 
-        conf.put(Config.TOPOLOGY_ACKER_RESOURCES_ONHEAP_MEMORY_MB, 250);
-        conf.put(Config.TOPOLOGY_ACKER_CPU_PCORE_PERCENT, 50);
+            conf.put(Config.TOPOLOGY_ACKER_RESOURCES_ONHEAP_MEMORY_MB, 250);
+            conf.put(Config.TOPOLOGY_ACKER_CPU_PCORE_PERCENT, 50);
 
-        TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormTopology, 0,
-                genExecsAndComps(StormCommon.systemTopology(conf, stormTopology)), currentTime, "user");
+            TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormTopology, 0,
+                    genExecsAndComps(StormCommon.systemTopology(conf, stormTopology)), currentTime, "user");
 
-        Topologies topologies = new Topologies(topo);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
+            Topologies topologies = new Topologies(topo);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
 
-        scheduler = new ResourceAwareScheduler();
+            scheduler = new ResourceAwareScheduler();
 
-        scheduler.prepare(conf, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
+            scheduler.prepare(conf, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
 
-        // We need to have 3 slots on 3 separate hosts. The topology needs 6 GPUs 3500 MB memory and 350% CPU
-        // The bolt-3 instances must be on separate nodes because they each need 2 GPUs.
-        // The bolt-2 instances must be on the same node as they each need 1 GPU
-        // (this assumes that we are packing the components to avoid fragmentation).
-        // The bolt-1 and spout instances fill in the rest.
+            // We need to have 3 slots on 3 separate hosts. The topology needs 6 GPUs 3500 MB memory and 350% CPU
+            // The bolt-3 instances must be on separate nodes because they each need 2 GPUs.
+            // The bolt-2 instances must be on the same node as they each need 1 GPU
+            // (this assumes that we are packing the components to avoid fragmentation).
+            // The bolt-1 and spout instances fill in the rest.
 
-        // Ordered execs: [[6, 6], [2, 2], [4, 4], [5, 5], [1, 1], [3, 3], [0, 0]]
-        // Ackers: [[8, 8], [7, 7]] (+ [[9, 9], [10, 10]] when numOfAckersPerWorker=2)
-        HashSet<HashSet<ExecutorDetails>> expectedScheduling = new HashSet<>();
-        if (numOfAckersPerWorker == -1 || numOfAckersPerWorker == 1) {
-            expectedScheduling.add(new HashSet<>(Collections.singletonList(
-                new ExecutorDetails(3, 3)))); //bolt-3 - 500 MB, 50% CPU, 2 GPU
-            //Total 500 MB, 50% CPU, 2 - GPU -> this node has 1500 MB, 150% cpu, 0 GPU left
-            expectedScheduling.add(new HashSet<>(Arrays.asList(
-                new ExecutorDetails(6, 6), //bolt-2 - 500 MB, 50% CPU, 1 GPU
-                new ExecutorDetails(2, 2), //bolt-1 - 500 MB, 50% CPU, 0 GPU
-                new ExecutorDetails(5, 5), //bolt-2 - 500 MB, 50% CPU, 1 GPU
-                new ExecutorDetails(8, 8)))); //acker - 250 MB, 50% CPU, 0 GPU
-            //Total 1750 MB, 200% CPU, 2 GPU -> this node has 250 MB, 0% CPU, 0 GPU left
-            expectedScheduling.add(new HashSet<>(Arrays.asList(
-                new ExecutorDetails(4, 4), //bolt-3 500 MB, 50% cpu, 2 GPU
-                new ExecutorDetails(1, 1), //bolt-1 - 500 MB, 50% CPU, 0 GPU
-                new ExecutorDetails(0, 0), //Spout - 500 MB, 50% CPU, 0 GPU
-                new ExecutorDetails(7, 7) ))); //acker - 250 MB, 50% CPU, 0 GPU
-            //Total 1750 MB, 200% CPU, 2 GPU -> this node has 250 MB, 0% CPU, 0 GPU left
-        } else if (numOfAckersPerWorker == 0) {
-            expectedScheduling.add(new HashSet<>(Collections.singletonList(
-                new ExecutorDetails(3, 3)))); //bolt-3 - 500 MB, 50% CPU, 2 GPU
-            //Total 500 MB, 50% CPU, 2 - GPU -> this node has 1500 MB, 150% cpu, 0 GPU left
-            expectedScheduling.add(new HashSet<>(Arrays.asList(
-                new ExecutorDetails(6, 6), //bolt-2 - 500 MB, 50% CPU, 1 GPU
-                new ExecutorDetails(2, 2), //bolt-1 - 500 MB, 50% CPU, 0 GPU
-                new ExecutorDetails(5, 5), //bolt-2 - 500 MB, 50% CPU, 1 GPU
-                new ExecutorDetails(1, 1))));  //bolt-1 - 500 MB, 50% CPU, 0 GPU
-            //Total 2000 MB, 200% CPU, 2 GPU -> this node has 0 MB, 0% CPU, 0 GPU left
-            expectedScheduling.add(new HashSet<>(Arrays.asList(
-                new ExecutorDetails(0, 0), //Spout - 500 MB, 50% CPU, 0 GPU
-                new ExecutorDetails(4, 4)))); //bolt-3 500 MB, 50% cpu, 2 GPU
-            //Total 1000 MB, 100% CPU, 2 GPU -> this node has 1000 MB, 100% CPU, 0 GPU left
-        } else if (numOfAckersPerWorker == 2) {
+            // Ordered execs: [[6, 6], [2, 2], [4, 4], [5, 5], [1, 1], [3, 3], [0, 0]]
+            // Ackers: [[8, 8], [7, 7]] (+ [[9, 9], [10, 10]] when numOfAckersPerWorker=2)
+            HashSet<HashSet<ExecutorDetails>> expectedScheduling = new HashSet<>();
             expectedScheduling.add(new HashSet<>(Collections.singletonList(
-                new ExecutorDetails(3, 3)))); //bolt-3 - 500 MB, 50% CPU, 2 GPU
+                    new ExecutorDetails(3, 3)))); //bolt-3 - 500 MB, 50% CPU, 2 GPU
             //Total 500 MB, 50% CPU, 2 - GPU -> this node has 1500 MB, 150% cpu, 0 GPU left
             expectedScheduling.add(new HashSet<>(Arrays.asList(
-                new ExecutorDetails(7, 7),      //acker - 250 MB, 50% CPU, 0 GPU
-                new ExecutorDetails(8, 8),      //acker - 250 MB, 50% CPU, 0 GPU
-                new ExecutorDetails(6, 6),      //bolt-2 - 500 MB, 50% CPU, 1 GPU
-                new ExecutorDetails(2, 2))));   //bolt-1 - 500 MB, 50% CPU, 0 GPU
+                    new ExecutorDetails(7, 7),      //acker - 250 MB, 50% CPU, 0 GPU
+                    new ExecutorDetails(8, 8),      //acker - 250 MB, 50% CPU, 0 GPU
+                    new ExecutorDetails(6, 6),      //bolt-2 - 500 MB, 50% CPU, 1 GPU
+                    new ExecutorDetails(2, 2))));   //bolt-1 - 500 MB, 50% CPU, 0 GPU
             //Total 1500 MB, 200% CPU, 2 GPU -> this node has 500 MB, 0% CPU, 0 GPU left
             expectedScheduling.add(new HashSet<>(Arrays.asList(
-                new ExecutorDetails(9, 9),    //acker- 250 MB, 50% CPU, 0 GPU
-                new ExecutorDetails(10, 10),  //acker- 250 MB, 50% CPU, 0 GPU
-                new ExecutorDetails(1, 1),    //bolt-1 - 500 MB, 50% CPU, 0 GPU
-                new ExecutorDetails(4, 4)))); //bolt-3 500 MB, 50% cpu, 2 GPU
+                    new ExecutorDetails(9, 9),    //acker- 250 MB, 50% CPU, 0 GPU
+                    new ExecutorDetails(10, 10),  //acker- 250 MB, 50% CPU, 0 GPU
+                    new ExecutorDetails(1, 1),    //bolt-1 - 500 MB, 50% CPU, 0 GPU
+                    new ExecutorDetails(4, 4)))); //bolt-3 500 MB, 50% cpu, 2 GPU
             //Total 1500 MB, 200% CPU, 2 GPU -> this node has 500 MB, 0% CPU, 0 GPU left
             expectedScheduling.add(new HashSet<>(Arrays.asList(
-                new ExecutorDetails(0, 0), //Spout - 500 MB, 50% CPU, 0 GPU
-                new ExecutorDetails(5, 5)))); //bolt-2 - 500 MB, 50% CPU, 1 GPU
+                    new ExecutorDetails(0, 0), //Spout - 500 MB, 50% CPU, 0 GPU
+                    new ExecutorDetails(5, 5)))); //bolt-2 - 500 MB, 50% CPU, 1 GPU
             //Total 1000 MB, 100% CPU, 2 GPU -> this node has 1000 MB, 100% CPU, 0 GPU left
-        }
-        HashSet<HashSet<ExecutorDetails>> foundScheduling = new HashSet<>();
-        SchedulerAssignment assignment = cluster.getAssignmentById("testTopology-id");
-        for (Collection<ExecutorDetails> execs : assignment.getSlotToExecutors().values()) {
-            foundScheduling.add(new HashSet<>(execs));
-        }
 
-        assertEquals(expectedScheduling, foundScheduling);
-    }
-
-    /**
-     * Test if the scheduling logic for the GenericResourceAwareStrategy is correct
-     * with setting {@link Config#TOPOLOGY_ACKER_EXECUTORS}.
-     *
-     * Test details refer to {@link TestDefaultResourceAwareStrategy#testDefaultResourceAwareStrategyWithSettingAckerExecutors(int)}
-     */
-    @ParameterizedTest
-    @ValueSource(ints = {-1, 0, 2, 200})
-    public void testGenericResourceAwareStrategyWithSettingAckerExecutors(int numOfAckersPerWorker)
-        throws InvalidTopologyException {
-        int spoutParallelism = 1;
-        int boltParallelism = 2;
-        TopologyBuilder builder = new TopologyBuilder();
-        builder.setSpout("spout", new TestSpout(),
-            spoutParallelism);
-        builder.setBolt("bolt-1", new TestBolt(),
-            boltParallelism).shuffleGrouping("spout");
-        builder.setBolt("bolt-2", new TestBolt(),
-            boltParallelism).shuffleGrouping("bolt-1").addResource("gpu.count", 1.0);
-        builder.setBolt("bolt-3", new TestBolt(),
-            boltParallelism).shuffleGrouping("bolt-2").addResource("gpu.count", 2.0);
-
-        String topoName = "testTopology";
-        StormTopology stormTopology = builder.createTopology();
-
-        INimbus iNimbus = new INimbusTest();
-
-        Config conf = createGrasClusterConfig(50, 500, 0, null, Collections.emptyMap());
-        Map<String, Double> genericResourcesMap = new HashMap<>();
-        genericResourcesMap.put("gpu.count", 2.0);
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 200, 2000, genericResourcesMap);
-
-
-        conf.put(Config.TOPOLOGY_PRIORITY, 0);
-        conf.put(Config.TOPOLOGY_NAME, topoName);
-        conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 2000);
-        conf.put(Config.TOPOLOGY_SUBMITTER_USER, "user");
-
-        conf.put(Config.TOPOLOGY_ACKER_EXECUTORS, 4);
-        if (numOfAckersPerWorker == -1) {
-            // Leave topology.acker.executors.per.worker unset
-        } else {
-            conf.put(Config.TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER, numOfAckersPerWorker);
-        }
+            HashSet<HashSet<ExecutorDetails>> foundScheduling = new HashSet<>();
+            SchedulerAssignment assignment = cluster.getAssignmentById("testTopology-id");
+            for (Collection<ExecutorDetails> execs : assignment.getSlotToExecutors().values()) {
+                foundScheduling.add(new HashSet<>(execs));
+            }
 
-        int estimatedNumWorker = ServerUtils.getEstimatedWorkerCountForRasTopo(conf, stormTopology);
-        Nimbus.setUpAckerExecutorConfigs(topoName, conf, conf, estimatedNumWorker);
-
-        conf.put(Config.TOPOLOGY_ACKER_RESOURCES_ONHEAP_MEMORY_MB, 250);
-        conf.put(Config.TOPOLOGY_ACKER_CPU_PCORE_PERCENT, 50);
-
-        TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormTopology, 0,
-            genExecsAndComps(StormCommon.systemTopology(conf, stormTopology)), currentTime, "user");
-
-        Topologies topologies = new Topologies(topo);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
-
-        scheduler = new ResourceAwareScheduler();
-
-        scheduler.prepare(conf, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-
-        // We need to have 3 slots on 3 separate hosts. The topology needs 6 GPUs 3500 MB memory and 350% CPU
-        // The bolt-3 instances must be on separate nodes because they each need 2 GPUs.
-        // The bolt-2 instances must be on the same node as they each need 1 GPU
-        // (this assumes that we are packing the components to avoid fragmentation).
-        // The bolt-1 and spout instances fill in the rest.
-
-        // Ordered execs: [[6, 6], [2, 2], [4, 4], [5, 5], [1, 1], [3, 3], [0, 0]]
-        // Ackers: [[8, 8], [7, 7]] (+ [[9, 9], [10, 10]] when numOfAckersPerWorker=2)
-        HashSet<HashSet<ExecutorDetails>> expectedScheduling = new HashSet<>();
-        expectedScheduling.add(new HashSet<>(Collections.singletonList(
-            new ExecutorDetails(3, 3)))); //bolt-3 - 500 MB, 50% CPU, 2 GPU
-        //Total 500 MB, 50% CPU, 2 - GPU -> this node has 1500 MB, 150% cpu, 0 GPU left
-        expectedScheduling.add(new HashSet<>(Arrays.asList(
-            new ExecutorDetails(7, 7),      //acker - 250 MB, 50% CPU, 0 GPU
-            new ExecutorDetails(8, 8),      //acker - 250 MB, 50% CPU, 0 GPU
-            new ExecutorDetails(6, 6),      //bolt-2 - 500 MB, 50% CPU, 1 GPU
-            new ExecutorDetails(2, 2))));   //bolt-1 - 500 MB, 50% CPU, 0 GPU
-        //Total 1500 MB, 200% CPU, 2 GPU -> this node has 500 MB, 0% CPU, 0 GPU left
-        expectedScheduling.add(new HashSet<>(Arrays.asList(
-            new ExecutorDetails(9, 9),    //acker- 250 MB, 50% CPU, 0 GPU
-            new ExecutorDetails(10, 10),  //acker- 250 MB, 50% CPU, 0 GPU
-            new ExecutorDetails(1, 1),    //bolt-1 - 500 MB, 50% CPU, 0 GPU
-            new ExecutorDetails(4, 4)))); //bolt-3 500 MB, 50% cpu, 2 GPU
-        //Total 1500 MB, 200% CPU, 2 GPU -> this node has 500 MB, 0% CPU, 0 GPU left
-        expectedScheduling.add(new HashSet<>(Arrays.asList(
-            new ExecutorDetails(0, 0), //Spout - 500 MB, 50% CPU, 0 GPU
-            new ExecutorDetails(5, 5)))); //bolt-2 - 500 MB, 50% CPU, 1 GPU
-        //Total 1000 MB, 100% CPU, 2 GPU -> this node has 1000 MB, 100% CPU, 0 GPU left
-
-        HashSet<HashSet<ExecutorDetails>> foundScheduling = new HashSet<>();
-        SchedulerAssignment assignment = cluster.getAssignmentById("testTopology-id");
-        for (Collection<ExecutorDetails> execs : assignment.getSlotToExecutors().values()) {
-            foundScheduling.add(new HashSet<>(execs));
+            assertEquals(expectedScheduling, foundScheduling);
         }
-
-        assertEquals(expectedScheduling, foundScheduling);
     }
 
     private TopologyDetails createTestStormTopology(StormTopology stormTopology, int priority, String name, Config conf) {
@@ -430,56 +436,58 @@ public class TestGenericResourceAwareStrategy {
      */
     @Test
     public void testGrasRequiringEviction() {
-        int spoutParallelism = 3;
-        double cpuPercent = 10;
-        double memoryOnHeap = 10;
-        double memoryOffHeap = 10;
-        // Sufficient Cpu/Memory. But insufficient gpu to schedule all topologies (gpu1, noGpu, gpu2).
-
-        // gpu topology (requires 3 gpu's in total)
-        TopologyBuilder builder = new TopologyBuilder();
-        builder.setSpout("spout", new TestSpout(), spoutParallelism).addResource("gpu.count", 1.0);
-        StormTopology stormTopologyWithGpu = builder.createTopology();
-
-        // non-gpu topology
-        builder = new TopologyBuilder();
-        builder.setSpout("spout", new TestSpout(), spoutParallelism);
-        StormTopology stormTopologyNoGpu = builder.createTopology();
-
-        Config conf = createGrasClusterConfig(cpuPercent, memoryOnHeap, memoryOffHeap, null, Collections.emptyMap());
-        conf.put(DaemonConfig.RESOURCE_AWARE_SCHEDULER_MAX_TOPOLOGY_SCHEDULING_ATTEMPTS, 2);    // allow 1 round of evictions
-
-        String gpu1 = "hasGpu1";
-        String noGpu = "hasNoGpu";
-        String gpu2 = "hasGpu2";
-        TopologyDetails topo[] = {
-                createTestStormTopology(stormTopologyWithGpu, 10, gpu1, conf),
-                createTestStormTopology(stormTopologyNoGpu, 10, noGpu, conf),
-                createTestStormTopology(stormTopologyWithGpu, 9, gpu2, conf)
-        };
-        Topologies topologies = new Topologies(topo[0], topo[1]);
-
-        Map<String, Double> genericResourcesMap = new HashMap<>();
-        genericResourcesMap.put("gpu.count", 1.0);
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 500, 2000, genericResourcesMap);
-        Cluster cluster = new Cluster(new INimbusTest(), new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
-
-        // should schedule gpu1 and noGpu successfully
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(conf, new StormMetricsRegistry());
-        scheduler.schedule(topologies, cluster);
-        assertTopologiesFullyScheduled(cluster, gpu1);
-        assertTopologiesFullyScheduled(cluster, noGpu);
-
-        // should evict gpu1 and noGpu topologies in order to schedule gpu2 topology; then fail to reschedule gpu1 topology;
-        // then schedule noGpu topology.
-        // Scheduling used to ignore gpu resource when deciding when to stop evicting, and gpu2 would fail to schedule.
-        topologies = new Topologies(topo[0], topo[1], topo[2]);
-        cluster = new Cluster(cluster, topologies);
-        scheduler.schedule(topologies, cluster);
-        assertTopologiesNotScheduled(cluster, gpu1);
-        assertTopologiesFullyScheduled(cluster, noGpu);
-        assertTopologiesFullyScheduled(cluster, gpu2);
+        for (Class strategyClass: strategyClasses) {
+            int spoutParallelism = 3;
+            double cpuPercent = 10;
+            double memoryOnHeap = 10;
+            double memoryOffHeap = 10;
+            // Sufficient Cpu/Memory. But insufficient gpu to schedule all topologies (gpu1, noGpu, gpu2).
+
+            // gpu topology (requires 3 gpu's in total)
+            TopologyBuilder builder = new TopologyBuilder();
+            builder.setSpout("spout", new TestSpout(), spoutParallelism).addResource("gpu.count", 1.0);
+            StormTopology stormTopologyWithGpu = builder.createTopology();
+
+            // non-gpu topology
+            builder = new TopologyBuilder();
+            builder.setSpout("spout", new TestSpout(), spoutParallelism);
+            StormTopology stormTopologyNoGpu = builder.createTopology();
+
+            Config conf = createGrasClusterConfig(strategyClass, cpuPercent, memoryOnHeap, memoryOffHeap, null, Collections.emptyMap());
+            conf.put(DaemonConfig.RESOURCE_AWARE_SCHEDULER_MAX_TOPOLOGY_SCHEDULING_ATTEMPTS, 2);    // allow 1 round of evictions
+
+            String gpu1 = "hasGpu1";
+            String noGpu = "hasNoGpu";
+            String gpu2 = "hasGpu2";
+            TopologyDetails topo[] = {
+                    createTestStormTopology(stormTopologyWithGpu, 10, gpu1, conf),
+                    createTestStormTopology(stormTopologyNoGpu, 10, noGpu, conf),
+                    createTestStormTopology(stormTopologyWithGpu, 9, gpu2, conf)
+            };
+            Topologies topologies = new Topologies(topo[0], topo[1]);
+
+            Map<String, Double> genericResourcesMap = new HashMap<>();
+            genericResourcesMap.put("gpu.count", 1.0);
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 500, 2000, genericResourcesMap);
+            Cluster cluster = new Cluster(new INimbusTest(), new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
+
+            // should schedule gpu1 and noGpu successfully
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(conf, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+            assertTopologiesFullyScheduled(cluster, strategyClass, gpu1);
+            assertTopologiesFullyScheduled(cluster, strategyClass, noGpu);
+
+            // should evict gpu1 and noGpu topologies in order to schedule gpu2 topology; then fail to reschedule gpu1 topology;
+            // then schedule noGpu topology.
+            // Scheduling used to ignore gpu resource when deciding when to stop evicting, and gpu2 would fail to schedule.
+            topologies = new Topologies(topo[0], topo[1], topo[2]);
+            cluster = new Cluster(cluster, topologies);
+            scheduler.schedule(topologies, cluster);
+            assertTopologiesNotScheduled(cluster, strategyClass, gpu1);
+            assertTopologiesFullyScheduled(cluster, strategyClass, noGpu);
+            assertTopologiesFullyScheduled(cluster, strategyClass, gpu2);
+        }
     }
     
     /**
@@ -488,124 +496,186 @@ public class TestGenericResourceAwareStrategy {
     @Test
     public void testGenericResourceAwareStrategyInFavorOfShuffle()
         throws InvalidTopologyException {
-        int spoutParallelism = 1;
-        int boltParallelism = 2;
-        TopologyBuilder builder = new TopologyBuilder();
-        builder.setSpout("spout", new TestSpout(),
-            spoutParallelism);
-        builder.setBolt("bolt-1", new TestBolt(),
-            boltParallelism).shuffleGrouping("spout");
-        builder.setBolt("bolt-2", new TestBolt(),
-            boltParallelism).shuffleGrouping("bolt-1").addResource("gpu.count", 1.0);
-        builder.setBolt("bolt-3", new TestBolt(),
-            boltParallelism).shuffleGrouping("bolt-2").addResource("gpu.count", 2.0);
-
-        StormTopology stormTopology = builder.createTopology();
-
-        INimbus iNimbus = new INimbusTest();
-
-        Config conf = createGrasClusterConfig(50, 250, 250, null, Collections.emptyMap());
-        Map<String, Double> genericResourcesMap = new HashMap<>();
-        genericResourcesMap.put("gpu.count", 2.0);
-        Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 200, 2000, genericResourcesMap);
-
-
-        conf.put(Config.TOPOLOGY_PRIORITY, 0);
-        conf.put(Config.TOPOLOGY_NAME, "testTopology");
-        conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE);
-        conf.put(Config.TOPOLOGY_SUBMITTER_USER, "user");
-        conf.put(Config.TOPOLOGY_RAS_ORDER_EXECUTORS_BY_PROXIMITY_NEEDS, true);
-
-        TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormTopology, 0,
-            genExecsAndComps(StormCommon.systemTopology(conf,stormTopology)), currentTime, "user");
-
-        Topologies topologies = new Topologies(topo);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
-
-        ResourceAwareScheduler rs = new ResourceAwareScheduler();
-
-        rs.prepare(conf, new StormMetricsRegistry());
-        rs.schedule(topologies, cluster);
-        // Sorted execs: [[0, 0], [2, 2], [6, 6], [4, 4], [1, 1], [5, 5], [3, 3], [7, 7]]
-        // Ackers: [[7, 7]]]
-
-        HashSet<HashSet<ExecutorDetails>> expectedScheduling = new HashSet<>();
-        expectedScheduling.add(new HashSet<>(Arrays.asList(
-            new ExecutorDetails(0, 0),      //spout
-            new ExecutorDetails(2, 2),      //bolt-1
-            new ExecutorDetails(6, 6),      //bolt-2
-            new ExecutorDetails(7, 7))));   //acker
-        expectedScheduling.add(new HashSet<>(Arrays.asList(
-            new ExecutorDetails(4, 4),      //bolt-3
-            new ExecutorDetails(1, 1))));   //bolt-1
-        expectedScheduling.add(new HashSet<>(Collections.singletonList(new ExecutorDetails(5, 5))));    //bolt-2
-        expectedScheduling.add(new HashSet<>(Collections.singletonList(new ExecutorDetails(3, 3))));    //bolt-3
-        HashSet<HashSet<ExecutorDetails>> foundScheduling = new HashSet<>();
-        SchedulerAssignment assignment = cluster.getAssignmentById("testTopology-id");
-        for (Collection<ExecutorDetails> execs : assignment.getSlotToExecutors().values()) {
-            foundScheduling.add(new HashSet<>(execs));
-        }
+        for (Class strategyClass: strategyClasses) {
+            int spoutParallelism = 1;
+            int boltParallelism = 2;
+            TopologyBuilder builder = new TopologyBuilder();
+            builder.setSpout("spout", new TestSpout(),
+                    spoutParallelism);
+            builder.setBolt("bolt-1", new TestBolt(),
+                    boltParallelism).shuffleGrouping("spout");
+            builder.setBolt("bolt-2", new TestBolt(),
+                    boltParallelism).shuffleGrouping("bolt-1").addResource("gpu.count", 1.0);
+            builder.setBolt("bolt-3", new TestBolt(),
+                    boltParallelism).shuffleGrouping("bolt-2").addResource("gpu.count", 2.0);
+
+            StormTopology stormTopology = builder.createTopology();
+
+            INimbus iNimbus = new INimbusTest();
+
+            Config conf = createGrasClusterConfig(strategyClass, 50, 250, 250, null, Collections.emptyMap());
+            Map<String, Double> genericResourcesMap = new HashMap<>();
+            genericResourcesMap.put("gpu.count", 2.0);
+            Map<String, SupervisorDetails> supMap = genSupervisors(4, 4, 200, 2000, genericResourcesMap);
+
+
+            conf.put(Config.TOPOLOGY_PRIORITY, 0);
+            conf.put(Config.TOPOLOGY_NAME, "testTopology");
+            conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE);
+            conf.put(Config.TOPOLOGY_SUBMITTER_USER, "user");
+            conf.put(Config.TOPOLOGY_RAS_ORDER_EXECUTORS_BY_PROXIMITY_NEEDS, true);
 
-        assertEquals(expectedScheduling, foundScheduling);
+            TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormTopology, 0,
+                    genExecsAndComps(StormCommon.systemTopology(conf, stormTopology)), currentTime, "user");
+
+            Topologies topologies = new Topologies(topo);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
+
+            ResourceAwareScheduler rs = new ResourceAwareScheduler();
+
+            rs.prepare(conf, new StormMetricsRegistry());
+            rs.schedule(topologies, cluster);
+            // Sorted execs: [[0, 0], [2, 2], [6, 6], [4, 4], [1, 1], [5, 5], [3, 3], [7, 7]]
+            // Ackers: [[7, 7]]]
+
+            HashSet<HashSet<ExecutorDetails>> expectedScheduling = new HashSet<>();
+            expectedScheduling.add(new HashSet<>(Arrays.asList(
+                    new ExecutorDetails(0, 0),      //spout
+                    new ExecutorDetails(2, 2),      //bolt-1
+                    new ExecutorDetails(6, 6),      //bolt-2
+                    new ExecutorDetails(7, 7))));   //acker
+            expectedScheduling.add(new HashSet<>(Arrays.asList(
+                    new ExecutorDetails(4, 4),      //bolt-3
+                    new ExecutorDetails(1, 1))));   //bolt-1
+            expectedScheduling.add(new HashSet<>(Collections.singletonList(new ExecutorDetails(5, 5))));    //bolt-2
+            expectedScheduling.add(new HashSet<>(Collections.singletonList(new ExecutorDetails(3, 3))));    //bolt-3
+            HashSet<HashSet<ExecutorDetails>> foundScheduling = new HashSet<>();
+            SchedulerAssignment assignment = cluster.getAssignmentById("testTopology-id");
+            for (Collection<ExecutorDetails> execs : assignment.getSlotToExecutors().values()) {
+                foundScheduling.add(new HashSet<>(execs));
+            }
+            assertEquals(expectedScheduling, foundScheduling);
+        }
     }
 
     @Test
     public void testAntiAffinityWithMultipleTopologies() {
-        INimbus iNimbus = new INimbusTest();
-        Map<String, SupervisorDetails> supMap = genSupervisorsWithRacks(1, 40, 66, 0, 0, 4700, 226200, new HashMap<>());
-        HashMap<String, Double> extraResources = new HashMap<>();
-        extraResources.put("my.gpu", 1.0);
-        supMap.putAll(genSupervisorsWithRacks(1, 40, 66, 1, 0, 4700, 226200, extraResources));
-
-        Config config = new Config();
-        config.putAll(createGrasClusterConfig(88, 775, 25, null, null));
-
-        scheduler = new ResourceAwareScheduler();
-        scheduler.prepare(config, new StormMetricsRegistry());
-
-        TopologyDetails tdSimple = genTopology("topology-simple", config, 1,
-            5, 100, 300, 0, 0, "user", 8192);
-
-        //Schedule the simple topology first
-        Topologies topologies = new Topologies(tdSimple);
-        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
-        scheduler.schedule(topologies, cluster);
-
-        TopologyBuilder builder = topologyBuilder(1, 5, 100, 300);
-        builder.setBolt("gpu-bolt", new TestBolt(), 40)
-            .addResource("my.gpu", 1.0)
-            .shuffleGrouping("spout-0");
-        TopologyDetails tdGpu = topoToTopologyDetails("topology-gpu", config, builder.createTopology(), 0, 0,"user", 8192);
-
-        //Now schedule GPU but with the simple topology in place.
-        topologies = new Topologies(tdSimple, tdGpu);
-        cluster = new Cluster(cluster, topologies);
-        scheduler.schedule(topologies, cluster);
-
-        Map<String, SchedulerAssignment> assignments = new TreeMap<>(cluster.getAssignments());
-        assertEquals(2, assignments.size());
-
-        Map<String, Map<String, AtomicLong>> topoPerRackCount = new HashMap<>();
-        for (Entry<String, SchedulerAssignment> entry: assignments.entrySet()) {
-            SchedulerAssignment sa = entry.getValue();
-            Map<String, AtomicLong> slotsPerRack = new TreeMap<>();
-            for (WorkerSlot slot : sa.getSlots()) {
-                String nodeId = slot.getNodeId();
-                String rack = supervisorIdToRackName(nodeId);
-                slotsPerRack.computeIfAbsent(rack, (r) -> new AtomicLong(0)).incrementAndGet();
+        for (Class strategyClass: strategyClasses) {
+            INimbus iNimbus = new INimbusTest();
+            Map<String, SupervisorDetails> supMap = genSupervisorsWithRacks(1, 40, 66, 0, 0, 4700, 226200, new HashMap<>());
+            HashMap<String, Double> extraResources = new HashMap<>();
+            extraResources.put("my.gpu", 1.0);
+            supMap.putAll(genSupervisorsWithRacks(1, 40, 66, 1, 0, 4700, 226200, extraResources));
+
+            Config config = new Config();
+            config.putAll(createGrasClusterConfig(strategyClass, 88, 775, 25, null, null));
+
+            scheduler = new ResourceAwareScheduler();
+            scheduler.prepare(config, new StormMetricsRegistry());
+
+            TopologyDetails tdSimple = genTopology("topology-simple", config, 1,
+                    5, 100, 300, 0, 0, "user", 8192);
+
+            //Schedule the simple topology first
+            Topologies topologies = new Topologies(tdSimple);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+            scheduler.schedule(topologies, cluster);
+
+            TopologyBuilder builder = topologyBuilder(1, 5, 100, 300);
+            builder.setBolt("gpu-bolt", new TestBolt(), 40)
+                    .addResource("my.gpu", 1.0)
+                    .shuffleGrouping("spout-0");
+            TopologyDetails tdGpu = topoToTopologyDetails("topology-gpu", config, builder.createTopology(), 0, 0, "user", 8192);
+
+            //Now schedule GPU but with the simple topology in place.
+            topologies = new Topologies(tdSimple, tdGpu);
+            cluster = new Cluster(cluster, topologies);
+            scheduler.schedule(topologies, cluster);
+
+            Map<String, SchedulerAssignment> assignments = new TreeMap<>(cluster.getAssignments());
+            assertEquals(2, assignments.size());
+
+            Map<String, Map<String, AtomicLong>> topoPerRackCount = new HashMap<>();
+            for (Entry<String, SchedulerAssignment> entry : assignments.entrySet()) {
+                SchedulerAssignment sa = entry.getValue();
+                Map<String, AtomicLong> slotsPerRack = new TreeMap<>();
+                for (WorkerSlot slot : sa.getSlots()) {
+                    String nodeId = slot.getNodeId();
+                    String rack = supervisorIdToRackName(nodeId);
+                    slotsPerRack.computeIfAbsent(rack, (r) -> new AtomicLong(0)).incrementAndGet();
+                }
+                LOG.info("{} => {}", entry.getKey(), slotsPerRack);
+                topoPerRackCount.put(entry.getKey(), slotsPerRack);
             }
-            LOG.info("{} => {}", entry.getKey(), slotsPerRack);
-            topoPerRackCount.put(entry.getKey(), slotsPerRack);
+
+            Map<String, AtomicLong> simpleCount = topoPerRackCount.get("topology-simple-0");
+            assertNotNull(simpleCount);
+            //Because the simple topology was scheduled first we want to be sure that it didn't put anything on
+            // the GPU nodes.
+            assertEquals(1, simpleCount.size()); //Only 1 rack is in use
+            assertFalse(simpleCount.containsKey("r001")); //r001 is the second rack with GPUs
+            assertTrue(simpleCount.containsKey("r000")); //r000 is the first rack with no GPUs
+
+            //We don't really care too much about the scheduling of topology-gpu-0, because it was scheduled.
         }
+    }
+
+    @Test
+    public void testScheduleLeftOverAckers() throws Exception {
+        for (Class strategyClass: strategyClasses) {
+            int spoutParallelism = 1;
+            TopologyBuilder builder = new TopologyBuilder();
+            builder.setSpout("spout", new TestSpout(), spoutParallelism);
+
+            String topoName = "testTopology";
+            StormTopology stormToplogy = builder.createTopology();
+
+            INimbus iNimbus = new INimbusTest();
+            Config conf = createGrasClusterConfig(strategyClass, 50, 400, 0, null, Collections.emptyMap());
+
+            Map<String, SupervisorDetails> supMap = genSupervisors(1, 1, 100, 1100);
+            Map<String, SupervisorDetails> tmpSupMap = genSupervisors(2, 1, 100, 400);
+            supMap.put("r000s001", tmpSupMap.get("r000s001"));
+            LOG.info("{}", tmpSupMap.get("r000s001"));
 
-        Map<String, AtomicLong> simpleCount = topoPerRackCount.get("topology-simple-0");
-        assertNotNull(simpleCount);
-        //Because the simple topology was scheduled first we want to be sure that it didn't put anything on
-        // the GPU nodes.
-        assertEquals(1, simpleCount.size()); //Only 1 rack is in use
-        assertFalse(simpleCount.containsKey("r001")); //r001 is the second rack with GPUs
-        assertTrue(simpleCount.containsKey("r000")); //r000 is the first rack with no GPUs
+            conf.put(Config.TOPOLOGY_PRIORITY, 0);
+            conf.put(Config.TOPOLOGY_NAME, topoName);
+            conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 1000);
+            conf.put(Config.TOPOLOGY_SUBMITTER_USER, "user");
+            conf.put(Config.TOPOLOGY_ACKER_EXECUTORS, 2);
+            conf.put(Config.TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER, 1);
 
-        //We don't really care too much about the scheduling of topology-gpu-0, because it was scheduled.
+            conf.put(Config.TOPOLOGY_ACKER_RESOURCES_ONHEAP_MEMORY_MB, 500);
+            conf.put(Config.TOPOLOGY_ACKER_CPU_PCORE_PERCENT, 0);
+
+            TopologyDetails topo = new TopologyDetails("testTopology-id", conf, stormToplogy, 0,
+                    genExecsAndComps(StormCommon.systemTopology(conf, stormToplogy)), currentTime, "user");
+
+            Topologies topologies = new Topologies(topo);
+            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
+
+            scheduler = new ResourceAwareScheduler();
+
+            scheduler.prepare(conf, new StormMetricsRegistry());
+            scheduler.schedule(topologies, cluster);
+
+            // First it tries too schedule spout [0, 0] with a bound acker [1, 1] to sup1 r000s000.
+            // However, sup2 r000s001 only has 400 on-heap mem which can not fit the left over acker [2, 2]
+            // So it backtrack to [0, 0] and put it to sup2 r000s001.
+            // Then put two ackers both as left-over ackers to sup1 r000s000.
+            HashSet<HashSet<ExecutorDetails>> expectedScheduling = new HashSet<>();
+            expectedScheduling.add(new HashSet<>(Arrays.asList(
+                    new ExecutorDetails(0, 0))));   // spout
+            expectedScheduling.add(new HashSet<>(Arrays.asList(
+                    new ExecutorDetails(1, 1),      // acker
+                    new ExecutorDetails(2, 2))));   // acker
+
+            HashSet<HashSet<ExecutorDetails>> foundScheduling = new HashSet<>();
+            SchedulerAssignment assignment = cluster.getAssignmentById("testTopology-id");
+            for (Collection<ExecutorDetails> execs : assignment.getSlotToExecutors().values()) {
+                foundScheduling.add(new HashSet<>(execs));
+            }
+            assertEquals(expectedScheduling, foundScheduling);
+        }
     }
 }
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestLargeCluster.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestLargeCluster.java
index 9c31f5fe0..ea15b2f62 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestLargeCluster.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestLargeCluster.java
@@ -69,6 +69,11 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
 @ExtendWith({NormalizedResourcesExtension.class})
 public class TestLargeCluster {
     private static final Logger LOG = LoggerFactory.getLogger(TestLargeCluster.class);
+    private static final Class[] strategyClasses = {
+            DefaultResourceAwareStrategy.class,
+            RoundRobinResourceAwareStrategy.class,
+            GenericResourceAwareStrategy.class,
+    };
 
     public enum TEST_CLUSTER_NAME {
         TEST_CLUSTER_01("largeCluster01"),
@@ -383,65 +388,67 @@ public class TestLargeCluster {
      */
     @Test
     public void testLargeCluster() throws Exception {
-        for (TEST_CLUSTER_NAME testClusterName: TEST_CLUSTER_NAME.values()) {
-            LOG.info("********************************************");
-            LOG.info("testLargeCluster: Start Processing cluster {}", testClusterName.getClusterName());
-
-            String resourcePath = testClusterName.getResourcePath();
-            Map<String, SupervisorDetails> supervisors = createSupervisors(testClusterName, 0);
-
-            TopologyDetails[] topoDetailsArray = createTopoDetailsArray(resourcePath, false);
-            assertTrue(topoDetailsArray.length > 0, "No topologies found for cluster " + testClusterName.getClusterName());
-            Topologies topologies = new Topologies(topoDetailsArray);
-
-            Config confWithDefaultStrategy = new Config();
-            confWithDefaultStrategy.putAll(topoDetailsArray[0].getConf());
-            confWithDefaultStrategy.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, DefaultResourceAwareStrategy.class.getName());
-            confWithDefaultStrategy.put(
-                Config.STORM_NETWORK_TOPOGRAPHY_PLUGIN,
-                TestUtilsForResourceAwareScheduler.GenSupervisorsDnsToSwitchMapping.class.getName());
-
-            INimbus iNimbus = new INimbusTest();
-            Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supervisors, new HashMap<>(),
-                topologies, confWithDefaultStrategy);
-
-            scheduler = new ResourceAwareScheduler();
+        for (Class strategyClass: strategyClasses) {
+            for (TEST_CLUSTER_NAME testClusterName : TEST_CLUSTER_NAME.values()) {
+                LOG.info("********************************************");
+                LOG.info("testLargeCluster: Start Processing cluster {} using ", testClusterName.getClusterName(), strategyClass.getName());
+
+                String resourcePath = testClusterName.getResourcePath();
+                Map<String, SupervisorDetails> supervisors = createSupervisors(testClusterName, 0);
+
+                TopologyDetails[] topoDetailsArray = createTopoDetailsArray(resourcePath, false);
+                assertTrue(topoDetailsArray.length > 0, "No topologies found for cluster " + testClusterName.getClusterName());
+                Topologies topologies = new Topologies(topoDetailsArray);
+
+                Config confWithDefaultStrategy = new Config();
+                confWithDefaultStrategy.putAll(topoDetailsArray[0].getConf());
+                confWithDefaultStrategy.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, strategyClass.getName());
+                confWithDefaultStrategy.put(
+                        Config.STORM_NETWORK_TOPOGRAPHY_PLUGIN,
+                        TestUtilsForResourceAwareScheduler.GenSupervisorsDnsToSwitchMapping.class.getName());
+
+                INimbus iNimbus = new INimbusTest();
+                Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supervisors, new HashMap<>(),
+                        topologies, confWithDefaultStrategy);
+
+                scheduler = new ResourceAwareScheduler();
+
+                List<Class> classesToDebug = Arrays.asList(DefaultResourceAwareStrategy.class,
+                        GenericResourceAwareStrategy.class, ResourceAwareScheduler.class,
+                        Cluster.class
+                );
+                Level logLevel = Level.INFO; // switch to Level.DEBUG for verbose otherwise Level.INFO
+                classesToDebug.forEach(x -> Configurator.setLevel(x.getName(), logLevel));
+                long startTime = System.currentTimeMillis();
+                scheduler.prepare(confWithDefaultStrategy, new StormMetricsRegistry());
+                scheduler.schedule(topologies, cluster);
+                long endTime = System.currentTimeMillis();
+                LOG.info("Cluster={} Scheduling Time: {} topologies in {} seconds",
+                        testClusterName.getClusterName(), topoDetailsArray.length, (endTime - startTime) / 1000.0);
+
+                for (TopologyDetails td : topoDetailsArray) {
+                    TestUtilsForResourceAwareScheduler.assertTopologiesFullyScheduled(cluster, strategyClass, td.getName());
+                }
 
-            List<Class> classesToDebug = Arrays.asList(DefaultResourceAwareStrategy.class,
-                GenericResourceAwareStrategy.class, ResourceAwareScheduler.class,
-                Cluster.class
-            );
-            Level logLevel = Level.INFO ; // switch to Level.DEBUG for verbose otherwise Level.INFO
-            classesToDebug.forEach(x -> Configurator.setLevel(x.getName(), logLevel));
-            long startTime = System.currentTimeMillis();
-            scheduler.prepare(confWithDefaultStrategy, new StormMetricsRegistry());
-            scheduler.schedule(topologies, cluster);
-            long endTime = System.currentTimeMillis();
-            LOG.info("Cluster={} Scheduling Time: {} topologies in {} seconds",
-                testClusterName.getClusterName(), topoDetailsArray.length, (endTime - startTime) / 1000.0);
-
-            for (TopologyDetails td : topoDetailsArray) {
-                TestUtilsForResourceAwareScheduler.assertTopologiesFullyScheduled(cluster, td.getName());
-            }
+                // Remove topology and reschedule it
+                for (int i = 0; i < topoDetailsArray.length; i++) {
+                    startTime = System.currentTimeMillis();
+                    TopologyDetails topoDetails = topoDetailsArray[i];
+                    cluster.unassign(topoDetails.getId());
+                    LOG.info("Cluster={},  ({}) Removed topology {}", testClusterName.getClusterName(), i, topoDetails.getName());
+                    IScheduler rescheduler = new ResourceAwareScheduler();
+                    rescheduler.prepare(confWithDefaultStrategy, new StormMetricsRegistry());
+                    rescheduler.schedule(topologies, cluster);
+                    TestUtilsForResourceAwareScheduler.assertTopologiesFullyScheduled(cluster, strategyClass, topoDetails.getName());
+                    endTime = System.currentTimeMillis();
+                    LOG.info("Cluster={}, ({}) Scheduling Time: Removed topology {} and rescheduled in {} seconds",
+                            testClusterName.getClusterName(), i, topoDetails.getName(), (endTime - startTime) / 1000.0);
+                }
+                classesToDebug.forEach(x -> Configurator.setLevel(x.getName(), Level.INFO));
 
-            // Remove topology and reschedule it
-            for (int i = 0 ; i < topoDetailsArray.length ; i++) {
-                startTime = System.currentTimeMillis();
-                TopologyDetails topoDetails = topoDetailsArray[i];
-                cluster.unassign(topoDetails.getId());
-                LOG.info("Cluster={},  ({}) Removed topology {}", testClusterName.getClusterName(), i, topoDetails.getName());
-                IScheduler rescheduler = new ResourceAwareScheduler();
-                rescheduler.prepare(confWithDefaultStrategy, new StormMetricsRegistry());
-                rescheduler.schedule(topologies, cluster);
-                TestUtilsForResourceAwareScheduler.assertTopologiesFullyScheduled(cluster, topoDetails.getName());
-                endTime = System.currentTimeMillis();
-                LOG.info("Cluster={}, ({}) Scheduling Time: Removed topology {} and rescheduled in {} seconds",
-                    testClusterName.getClusterName(), i, topoDetails.getName(), (endTime - startTime) / 1000.0);
+                LOG.info("testLargeCluster: End Processing cluster {}", testClusterName.getClusterName());
+                LOG.info("********************************************");
             }
-            classesToDebug.forEach(x -> Configurator.setLevel(x.getName(), Level.INFO));
-
-            LOG.info("testLargeCluster: End Processing cluster {}", testClusterName.getClusterName());
-            LOG.info("********************************************");
         }
     }
 
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestRoundRobinNodeSorterHostIsolation.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestRoundRobinNodeSorterHostIsolation.java
new file mode 100644
index 000000000..3d436a63d
--- /dev/null
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestRoundRobinNodeSorterHostIsolation.java
@@ -0,0 +1,265 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.storm.scheduler.resource.strategies.scheduling;
+
+import com.google.common.collect.Sets;
+import org.apache.storm.Config;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.networktopography.DNSToSwitchMapping;
+import org.apache.storm.scheduler.Cluster;
+import org.apache.storm.scheduler.INimbus;
+import org.apache.storm.scheduler.IScheduler;
+import org.apache.storm.scheduler.SchedulerAssignment;
+import org.apache.storm.scheduler.SupervisorDetails;
+import org.apache.storm.scheduler.Topologies;
+import org.apache.storm.scheduler.TopologyDetails;
+import org.apache.storm.scheduler.WorkerSlot;
+import org.apache.storm.scheduler.resource.ResourceAwareScheduler;
+import org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler;
+import org.apache.storm.scheduler.resource.normalization.NormalizedResourcesExtension;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.INimbusTest;
+import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.createRoundRobinClusterConfig;
+import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.genSupervisorsWithRacksAndNuma;
+import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.genTopology;
+import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.supervisorIdToRackName;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
+@ExtendWith({NormalizedResourcesExtension.class})
+public class TestRoundRobinNodeSorterHostIsolation {
+    private static final Logger LOG = LoggerFactory.getLogger(TestRoundRobinNodeSorterHostIsolation.class);
+    private static final int CURRENT_TIME = 1450418597;
+    private static final Class strategyClass = RoundRobinResourceAwareStrategy.class;
+
+    private Config createClusterConfig(double compPcore, double compOnHeap, double compOffHeap,
+                                       Map<String, Map<String, Number>> pools) {
+        Config config = TestUtilsForResourceAwareScheduler.createClusterConfig(compPcore, compOnHeap, compOffHeap, pools);
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, strategyClass.getName());
+        return config;
+    }
+
+    private static class TestDNSToSwitchMapping implements DNSToSwitchMapping {
+        private final Map<String, String> hostToRackMap;
+        private final Map<String, List<String>> rackToHosts;
+
+        @SafeVarargs
+        public TestDNSToSwitchMapping(Map<String, SupervisorDetails>... racks) {
+            Set<String> seenHosts = new HashSet<>();
+            Map<String, String> hostToRackMap = new HashMap<>();
+            Map<String, List<String>> rackToHosts = new HashMap<>();
+            for (int rackNum = 0; rackNum < racks.length; rackNum++) {
+                String rack = String.format("rack-%03d", rackNum);
+                for (SupervisorDetails sup : racks[rackNum].values()) {
+                    hostToRackMap.put(sup.getHost(), rack);
+                    String host = sup.getHost();
+                    if (!seenHosts.contains(host)) {
+                        rackToHosts.computeIfAbsent(rack, rid -> new ArrayList<>()).add(host);
+                        seenHosts.add(host);
+                    }
+                }
+            }
+            this.hostToRackMap = Collections.unmodifiableMap(hostToRackMap);
+            this.rackToHosts = Collections.unmodifiableMap(rackToHosts);
+        }
+
+        /**
+         * Use the "rack-%03d" embedded in the name of the supervisor to determine the rack number.
+         *
+         * @param supervisorDetailsCollection
+         */
+        public TestDNSToSwitchMapping(Collection<SupervisorDetails> supervisorDetailsCollection) {
+            Set<String> seenHosts = new HashSet<>();
+            Map<String, String> hostToRackMap = new HashMap<>();
+            Map<String, List<String>> rackToHosts = new HashMap<>();
+
+            for (SupervisorDetails supervisorDetails: supervisorDetailsCollection) {
+                String rackId = supervisorIdToRackName(supervisorDetails.getId());
+                hostToRackMap.put(supervisorDetails.getHost(), rackId);
+                String host = supervisorDetails.getHost();
+                if (!seenHosts.contains(host)) {
+                    rackToHosts.computeIfAbsent(rackId, rid -> new ArrayList<>()).add(host);
+                    seenHosts.add(host);
+                }
+            }
+            this.hostToRackMap = Collections.unmodifiableMap(hostToRackMap);
+            this.rackToHosts = Collections.unmodifiableMap(rackToHosts);
+        }
+
+        @Override
+        public Map<String, String> resolve(List<String> names) {
+            return hostToRackMap;
+        }
+
+        public Map<String, List<String>> getRackToHosts() {
+            return rackToHosts;
+        }
+    }
+
+    /**
+     * Free one-fifth of WorkerSlots.
+     */
+    private void freeSomeWorkerSlots(Cluster cluster) {
+        Map<String, SchedulerAssignment> assignmentMap = cluster.getAssignments();
+        for (SchedulerAssignment schedulerAssignment: assignmentMap.values()) {
+            int i = 0;
+            List<WorkerSlot> slotsToKill = new ArrayList<>();
+            for (WorkerSlot workerSlot: schedulerAssignment.getSlots()) {
+                i++;
+                if (i % 5 == 0) {
+                    slotsToKill.add(workerSlot);
+                }
+            }
+            cluster.freeSlots(slotsToKill);
+        }
+    }
+
+    /**
+     * Test whether number of nodes is limited by {@link Config#TOPOLOGY_ISOLATED_MACHINES} by scheduling
+     * two topologies and verifying the number of nodes that each one occupies and are not overlapping.
+     */
+    @Test
+    void testTopologyIsolation() {
+        INimbus iNimbus = new INimbusTest();
+        double compPcore = 100;
+        double compOnHeap = 775;
+        double compOffHeap = 25;
+        int[] topoNumSpouts = {1,1};
+        int[] topoNumBolts = {1,1};
+        int[] topoSpoutParallelism = {100, 100};
+        int[] topoBoltParallelism = {200, 200};
+        final int numRacks = 3;
+        final int numSupersPerRack = 10;
+        final int numPortsPerSuper = 6;
+        final int numZonesPerHost = 1;
+        final double numaResourceMultiplier = 1.0;
+        int rackStartNum = 0;
+        int supStartNum = 0;
+        long compPerRack = (topoNumSpouts[0] * topoSpoutParallelism[0] + topoNumBolts[0] * topoBoltParallelism[0]
+                + topoNumSpouts[1] * topoSpoutParallelism[1]); // enough for topo1 but not topo1+topo2
+        long compPerSuper =  compPerRack / numSupersPerRack;
+        double cpuPerSuper = compPcore * compPerSuper;
+        double memPerSuper = (compOnHeap + compOffHeap) * compPerSuper;
+        double[] topoMaxHeapSize = {memPerSuper, memPerSuper};
+        final String[] topoNames = {"topology1", "topology2"};
+        int[] maxNodes = {15, 13};
+
+        Map<String, SupervisorDetails> supMap = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum, supStartNum,
+                cpuPerSuper, memPerSuper, Collections.emptyMap(), numaResourceMultiplier);
+        TestDNSToSwitchMapping testDNSToSwitchMapping = new TestDNSToSwitchMapping(supMap.values());
+
+        Config[] configs = new Config[topoNames.length];
+        TopologyDetails[] topos = new TopologyDetails[topoNames.length];
+        for (int i = 0 ; i < topoNames.length ; i++) {
+            configs[i] = new Config();
+            configs[i].putAll(createRoundRobinClusterConfig(compPcore, compOnHeap, compOffHeap, null, null));
+            configs[i].put(Config.TOPOLOGY_ISOLATED_MACHINES, maxNodes[i]);
+            topos[i] = genTopology(topoNames[i], configs[i], topoNumSpouts[i],
+                    topoNumBolts[i], topoSpoutParallelism[i], topoBoltParallelism[i], 0, 0, "user", topoMaxHeapSize[i]);
+        }
+        TopologyDetails td1 = topos[0];
+        TopologyDetails td2 = topos[1];
+
+        IScheduler scheduler = new ResourceAwareScheduler();
+        scheduler.prepare(configs[0], new StormMetricsRegistry());
+
+        //Schedule the topo1 topology and ensure it uses limited number of nodes
+        Topologies topologies = new Topologies(td1);
+        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, configs[0]);
+        cluster.setNetworkTopography(testDNSToSwitchMapping.getRackToHosts());
+
+        scheduler.schedule(topologies, cluster);
+        Set<String> assignedRacks = cluster.getAssignedRacks(topos[0].getId());
+        assertEquals(2 , assignedRacks.size(), "Racks for topology=" + td1.getId() + " is " + assignedRacks);
+
+        //Now schedule GPU but with the simple topology in place.
+        topologies = new Topologies(td1, td2);
+        cluster = new Cluster(cluster, topologies);
+        scheduler.schedule(topologies, cluster);
+
+        assignedRacks = cluster.getAssignedRacks(td1.getId(), td2.getId());
+        assertEquals(numRacks, assignedRacks.size(), "Racks for topologies=" + td1.getId() + "/" + td2.getId() + " is " + assignedRacks);
+
+        SchedulerAssignment[] assignments = new SchedulerAssignment[topoNames.length];
+        Collection<String>[] assignmentNodes = new Collection[topoNames.length];
+        for (int i = 0 ; i < topoNames.length ; i++) {
+            assignments[i] = cluster.getAssignmentById(topos[i].getId());
+            if (assignments[i] == null) {
+                fail("Topology " + topoNames[i] + " cannot be scheduled");
+            }
+            assignmentNodes[i] = assignments[i].getSlots().stream().map(WorkerSlot::getNodeId).collect(Collectors.toList());
+            assertEquals(maxNodes[i], assignmentNodes[i].size(), "Max Nodes for " + topoNames[i] + " assignment");
+        }
+        // confirm no overlap in nodes
+        Set<String> nodes1 = new HashSet<>(assignmentNodes[0]);
+        Set<String> nodes2 = new HashSet<>(assignmentNodes[1]);
+        Set<String> dupNodes = Sets.intersection(nodes1, nodes2);
+        if (dupNodes.size() > 0) {
+            List<String> lines = new ArrayList<>();
+            lines.add("Topologies shared nodes when not expected to");
+            lines.add("Duplicated nodes are " + String.join(",", dupNodes));
+            fail(String.join("\n\t", lines));
+        }
+        nodes2.removeAll(nodes1);
+
+        // topo2 gets scheduled on across the two racks even if there is one rack with enough capacity
+        assignedRacks = cluster.getAssignedRacks(td2.getId());
+        assertEquals(numRacks -1, assignedRacks.size(), "Racks for topologies=" + td2.getId() + " is " + assignedRacks);
+
+        // now unassign topo2, expect only two of three racks to be in use; free some slots and reschedule topo1 some topo1 executors
+        cluster.unassign(td2.getId());
+        assignedRacks = cluster.getAssignedRacks(td2.getId());
+        assertEquals(0, assignedRacks.size(),
+                "After unassigning topology " + td2.getId() + ", racks for topology=" + td2.getId() + " is " + assignedRacks);
+        assignedRacks = cluster.getAssignedRacks(td1.getId());
+        assertEquals(numRacks - 1, assignedRacks.size(),
+                "After unassigning topology " + td2.getId() + ", racks for topology=" + td1.getId() + " is " + assignedRacks);
+        assertFalse(cluster.needsSchedulingRas(td1),
+                "Topology " + td1.getId() + " should be fully assigned before freeing slots");
+        freeSomeWorkerSlots(cluster);
+        assertTrue(cluster.needsSchedulingRas(td1),
+                "Topology " + td1.getId() + " should need scheduling after freeing slots");
+
+        // then reschedule executors
+        scheduler.schedule(topologies, cluster);
+
+        // only two of three racks should be in use still
+        assignedRacks = cluster.getAssignedRacks(td1.getId());
+        assertEquals(numRacks - 1, assignedRacks.size(),
+                "After reassigning topology " + td2.getId() + ", racks for topology=" + td1.getId() + " is " + assignedRacks);
+    }
+}
\ No newline at end of file
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/TestRoundRobinNodeSorterHostProximity.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/TestRoundRobinNodeSorterHostProximity.java
new file mode 100644
index 000000000..c43ffa71a
--- /dev/null
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/TestRoundRobinNodeSorterHostProximity.java
@@ -0,0 +1,944 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.storm.scheduler.resource.strategies.scheduling.sorter;
+
+import org.apache.storm.Config;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.networktopography.DNSToSwitchMapping;
+import org.apache.storm.scheduler.Cluster;
+import org.apache.storm.scheduler.ExecutorDetails;
+import org.apache.storm.scheduler.INimbus;
+import org.apache.storm.scheduler.IScheduler;
+import org.apache.storm.scheduler.SchedulerAssignment;
+import org.apache.storm.scheduler.SupervisorDetails;
+import org.apache.storm.scheduler.Topologies;
+import org.apache.storm.scheduler.TopologyDetails;
+import org.apache.storm.scheduler.WorkerSlot;
+import org.apache.storm.scheduler.resource.RasNodes;
+import org.apache.storm.scheduler.resource.ResourceAwareScheduler;
+import org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler;
+import org.apache.storm.scheduler.resource.normalization.NormalizedResourceRequest;
+import org.apache.storm.scheduler.resource.normalization.NormalizedResourcesExtension;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
+import org.apache.storm.scheduler.resource.strategies.scheduling.BaseResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.ObjectResourcesItem;
+import org.apache.storm.scheduler.resource.strategies.scheduling.RoundRobinResourceAwareStrategy;
+import org.apache.storm.topology.TopologyBuilder;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.INimbusTest;
+import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.TestBolt;
+import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.createRoundRobinClusterConfig;
+import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.genSupervisorsWithRacks;
+import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.genSupervisorsWithRacksAndNuma;
+import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.genTopology;
+import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.supervisorIdToRackName;
+import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.topoToTopologyDetails;
+import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.topologyBuilder;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
+@ExtendWith({NormalizedResourcesExtension.class})
+public class TestRoundRobinNodeSorterHostProximity {
+    private static final Logger LOG = LoggerFactory.getLogger(TestRoundRobinNodeSorterHostProximity.class);
+    private static final int CURRENT_TIME = 1450418597;
+    private static final Class strategyClass = RoundRobinResourceAwareStrategy.class;
+
+    private Config createClusterConfig(double compPcore, double compOnHeap, double compOffHeap,
+                                       Map<String, Map<String, Number>> pools) {
+        Config config = TestUtilsForResourceAwareScheduler.createClusterConfig(compPcore, compOnHeap, compOffHeap, pools);
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, strategyClass.getName());
+        return config;
+    }
+
+    private static class TestDNSToSwitchMapping implements DNSToSwitchMapping {
+        private final Map<String, String> hostToRackMap;
+        private final Map<String, List<String>> rackToHosts;
+
+        @SafeVarargs
+        public TestDNSToSwitchMapping(Map<String, SupervisorDetails>... racks) {
+            Set<String> seenHosts = new HashSet<>();
+            Map<String, String> hostToRackMap = new HashMap<>();
+            Map<String, List<String>> rackToHosts = new HashMap<>();
+            for (int rackNum = 0; rackNum < racks.length; rackNum++) {
+                String rack = String.format("rack-%03d", rackNum);
+                for (SupervisorDetails sup : racks[rackNum].values()) {
+                    hostToRackMap.put(sup.getHost(), rack);
+                    String host = sup.getHost();
+                    if (!seenHosts.contains(host)) {
+                        rackToHosts.computeIfAbsent(rack, rid -> new ArrayList<>()).add(host);
+                        seenHosts.add(host);
+                    }
+                }
+            }
+            this.hostToRackMap = Collections.unmodifiableMap(hostToRackMap);
+            this.rackToHosts = Collections.unmodifiableMap(rackToHosts);
+        }
+
+        /**
+         * Use the "rack-%03d" embedded in the name of the supervisor to determine the rack number.
+         *
+         * @param supervisorDetailsCollection
+         */
+        public TestDNSToSwitchMapping(Collection<SupervisorDetails> supervisorDetailsCollection) {
+            Set<String> seenHosts = new HashSet<>();
+            Map<String, String> hostToRackMap = new HashMap<>();
+            Map<String, List<String>> rackToHosts = new HashMap<>();
+
+            for (SupervisorDetails supervisorDetails: supervisorDetailsCollection) {
+                String rackId = supervisorIdToRackName(supervisorDetails.getId());
+                hostToRackMap.put(supervisorDetails.getHost(), rackId);
+                String host = supervisorDetails.getHost();
+                if (!seenHosts.contains(host)) {
+                    rackToHosts.computeIfAbsent(rackId, rid -> new ArrayList<>()).add(host);
+                    seenHosts.add(host);
+                }
+            }
+            this.hostToRackMap = Collections.unmodifiableMap(hostToRackMap);
+            this.rackToHosts = Collections.unmodifiableMap(rackToHosts);
+        }
+
+        @Override
+        public Map<String, String> resolve(List<String> names) {
+            return hostToRackMap;
+        }
+
+        public Map<String, List<String>> getRackToHosts() {
+            return rackToHosts;
+        }
+    }
+
+    /**
+     * Test whether strategy will choose correct rack.
+     */
+    @Test
+    public void testMultipleRacksWithFavoritism() {
+        final Map<String, SupervisorDetails> supMap = new HashMap<>();
+        final int numRacks = 1;
+        final int numSupersPerRack = 10;
+        final int numPortsPerSuper = 4;
+        final int numZonesPerHost = 2;
+        int rackStartNum = 0;
+        int supStartNum = 0;
+        final Map<String, SupervisorDetails> supMapRack0 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                400, 8000, Collections.emptyMap(), 1.0);
+
+        //generate another rack of supervisors with less resources
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack1 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                200, 4000, Collections.emptyMap(), 1.0);
+
+        //generate some supervisors that are depleted of one resource
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack2 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                0, 8000, Collections.emptyMap(), 1.0);
+
+        //generate some that has a lot of memory but little of cpu
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack3 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                10, 8000 * 2 + 4000, Collections.emptyMap(), 1.0);
+
+        //generate some that has a lot of cpu but little of memory
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack4 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                400 + 200 + 10, 1000, Collections.emptyMap(), 1.0);
+
+        supMap.putAll(supMapRack0);
+        supMap.putAll(supMapRack1);
+        supMap.putAll(supMapRack2);
+        supMap.putAll(supMapRack3);
+        supMap.putAll(supMapRack4);
+
+        Config config = createClusterConfig(100, 500, 500, null);
+        config.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE);
+        INimbus iNimbus = new INimbusTest();
+
+        //create test DNSToSwitchMapping plugin
+        TestDNSToSwitchMapping testDNSToSwitchMapping =
+                new TestDNSToSwitchMapping(supMapRack0, supMapRack1, supMapRack2, supMapRack3, supMapRack4);
+
+        Config t1Conf = new Config();
+        t1Conf.putAll(config);
+        final List<String> t1FavoredHostNames = Arrays.asList("host-41", "host-42", "host-43");
+        t1Conf.put(Config.TOPOLOGY_SCHEDULER_FAVORED_NODES, t1FavoredHostNames);
+        final List<String> t1UnfavoredHostIds = Arrays.asList("host-1", "host-2", "host-3");
+        t1Conf.put(Config.TOPOLOGY_SCHEDULER_UNFAVORED_NODES, t1UnfavoredHostIds);
+        //generate topologies
+        TopologyDetails topo1 = genTopology("topo-1", t1Conf, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
+
+
+        Config t2Conf = new Config();
+        t2Conf.putAll(config);
+        t2Conf.put(Config.TOPOLOGY_SCHEDULER_FAVORED_NODES, Arrays.asList("host-31", "host-32", "host-33"));
+        t2Conf.put(Config.TOPOLOGY_SCHEDULER_UNFAVORED_NODES, Arrays.asList("host-11", "host-12", "host-13"));
+        TopologyDetails topo2 = genTopology("topo-2", t2Conf, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
+
+        Topologies topologies = new Topologies(topo1, topo2);
+        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+
+        List<String> supHostnames = new LinkedList<>();
+        for (SupervisorDetails sup : supMap.values()) {
+            supHostnames.add(sup.getHost());
+        }
+        Map<String, List<String>> rackToHosts = testDNSToSwitchMapping.getRackToHosts();
+        cluster.setNetworkTopography(rackToHosts);
+
+        NodeSorterHostProximity nodeSorter = new NodeSorterHostProximity(cluster, topo1, BaseResourceAwareStrategy.NodeSortType.COMMON);
+        nodeSorter.prepare(null);
+        List<ObjectResourcesItem> sortedRacks = StreamSupport.stream(nodeSorter.getSortedRacks().spliterator(), false)
+                .collect(Collectors.toList());
+        String rackSummaries = sortedRacks.stream()
+                .map(x -> String.format("Rack %s -> scheduled-cnt %d, min-avail %f, avg-avail %f, cpu %f, mem %f",
+                        x.id, nodeSorter.getScheduledExecCntByRackId().getOrDefault(x.id, new AtomicInteger(-1)).get(),
+                        x.minResourcePercent, x.avgResourcePercent,
+                        x.availableResources.getTotalCpu(),
+                        x.availableResources.getTotalMemoryMb()))
+                .collect(Collectors.joining("\n\t"));
+
+        Iterator<ObjectResourcesItem> it = sortedRacks.iterator();
+        // Ranked first since rack-000 has the most balanced set of resources
+        assertEquals("rack-004", it.next().id, "rack-004 should be ordered first\n\t" + rackSummaries);
+        // Ranked second since rack-1 has a balanced set of resources but less than rack-0
+        assertEquals("rack-000", it.next().id, "rack-000 should be ordered second\n\t" + rackSummaries);
+        // Ranked third since rack-4 has a lot of cpu but not a lot of memory
+        assertEquals("rack-003", it.next().id, "rack-003 should be ordered\n\t" + rackSummaries);
+        // Ranked fourth since rack-3 has alot of memory but not cpu
+        assertEquals("rack-001", it.next().id, "rack-001 should be ordered fourth\n\t" + rackSummaries);
+        //Ranked last since rack-2 has not cpu resources
+        assertEquals("rack-002", it.next().id, "rack-002 should be ordered fifth\n\t" + rackSummaries);
+    }
+
+    /**
+     * Test if hosts are presented together regardless of resource availability.
+     * Supervisors are created with multiple Numa zones in such a manner that resources on two numa zones on the same host
+     * differ widely in resource availability.
+     */
+    @Test
+    public void testMultipleRacksWithHostProximity() {
+        final Map<String, SupervisorDetails> supMap = new HashMap<>();
+        final int numRacks = 1;
+        final int numSupersPerRack = 12;
+        final int numPortsPerSuper = 4;
+        final int numZonesPerHost = 3;
+        final double numaResourceMultiplier = 0.4;
+        int rackStartNum = 0;
+        int supStartNum = 0;
+
+        final Map<String, SupervisorDetails> supMapRack0 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                400, 8000, Collections.emptyMap(), numaResourceMultiplier);
+
+        //generate another rack of supervisors with less resources
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack1 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                200, 4000, Collections.emptyMap(), numaResourceMultiplier);
+
+        //generate some supervisors that are depleted of one resource
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack2 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                0, 8000, Collections.emptyMap(), numaResourceMultiplier);
+
+        //generate some that has a lot of memory but little of cpu
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack3 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                10, 8000 * 2 + 4000, Collections.emptyMap(),numaResourceMultiplier);
+
+        //generate some that has a lot of cpu but little of memory
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack4 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                400 + 200 + 10, 1000, Collections.emptyMap(), numaResourceMultiplier);
+
+        supMap.putAll(supMapRack0);
+        supMap.putAll(supMapRack1);
+        supMap.putAll(supMapRack2);
+        supMap.putAll(supMapRack3);
+        supMap.putAll(supMapRack4);
+
+        Config config = createClusterConfig(100, 500, 500, null);
+        config.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE);
+        INimbus iNimbus = new INimbusTest();
+
+        //create test DNSToSwitchMapping plugin
+        TestDNSToSwitchMapping testDNSToSwitchMapping =
+                new TestDNSToSwitchMapping(supMapRack0, supMapRack1, supMapRack2, supMapRack3, supMapRack4);
+
+        Config t1Conf = new Config();
+        t1Conf.putAll(config);
+        final List<String> t1FavoredHostNames = Arrays.asList("host-41", "host-42", "host-43");
+        t1Conf.put(Config.TOPOLOGY_SCHEDULER_FAVORED_NODES, t1FavoredHostNames);
+        final List<String> t1UnfavoredHostIds = Arrays.asList("host-1", "host-2", "host-3");
+        t1Conf.put(Config.TOPOLOGY_SCHEDULER_UNFAVORED_NODES, t1UnfavoredHostIds);
+        //generate topologies
+        TopologyDetails topo1 = genTopology("topo-1", t1Conf, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
+
+
+        Config t2Conf = new Config();
+        t2Conf.putAll(config);
+        t2Conf.put(Config.TOPOLOGY_SCHEDULER_FAVORED_NODES, Arrays.asList("host-31", "host-32", "host-33"));
+        t2Conf.put(Config.TOPOLOGY_SCHEDULER_UNFAVORED_NODES, Arrays.asList("host-11", "host-12", "host-13"));
+        TopologyDetails topo2 = genTopology("topo-2", t2Conf, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
+
+        Topologies topologies = new Topologies(topo1, topo2);
+        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+
+        cluster.setNetworkTopography(testDNSToSwitchMapping.getRackToHosts());
+
+        INodeSorter nodeSorter = new NodeSorterHostProximity(cluster, topo1);
+        nodeSorter.prepare(null);
+
+        Set<String> seenHosts = new HashSet<>();
+        String prevHost = null;
+        List<String> errLines = new ArrayList();
+        Map<String, String> nodeToHost = new RasNodes(cluster).getNodeIdToHostname();
+        for (String nodeId: nodeSorter.sortAllNodes()) {
+            String host = nodeToHost.getOrDefault(nodeId, "no-host-for-node-" + nodeId);
+            errLines.add(String.format("\tnodeId:%s, host:%s", nodeId, host));
+            if (!host.equals(prevHost) && seenHosts.contains(host)) {
+                String err = String.format("Host %s for node %s is out of order:\n\t%s", host, nodeId, String.join("\n\t", errLines));
+                fail(err);
+            }
+            seenHosts.add(host);
+            prevHost = host;
+        }
+    }
+
+    /**
+     * Racks should be returned in order of decreasing capacity.
+     */
+    @Test
+    public void testMultipleRacksOrderedByCapacity() {
+        final Map<String, SupervisorDetails> supMap = new HashMap<>();
+        final int numRacks = 1;
+        final int numSupersPerRack = 10;
+        final int numPortsPerSuper = 4;
+        final int numZonesPerHost = 1;
+        final double numaResourceMultiplier = 1.0;
+        int rackStartNum = 0;
+        int supStartNum = 0;
+
+        final Map<String, SupervisorDetails> supMapRack0 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                600, 8000 - rackStartNum, Collections.emptyMap(), numaResourceMultiplier);
+
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack1 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                500, 8000 - rackStartNum, Collections.emptyMap(), numaResourceMultiplier);
+
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack2 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                400, 8000 - rackStartNum, Collections.emptyMap(), numaResourceMultiplier);
+
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack3 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                300, 8000 - rackStartNum, Collections.emptyMap(),numaResourceMultiplier);
+
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack4 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                200, 8000 - rackStartNum, Collections.emptyMap(), numaResourceMultiplier);
+
+        // too small to hold topology
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack5 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                100, 8000 - rackStartNum, Collections.singletonMap("gpu.count", 0.0), numaResourceMultiplier);
+
+        supMap.putAll(supMapRack0);
+        supMap.putAll(supMapRack1);
+        supMap.putAll(supMapRack2);
+        supMap.putAll(supMapRack3);
+        supMap.putAll(supMapRack4);
+        supMap.putAll(supMapRack5);
+
+        Config config = createClusterConfig(100, 500, 500, null);
+        config.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE);
+        INimbus iNimbus = new INimbusTest();
+
+        //create test DNSToSwitchMapping plugin
+        TestDNSToSwitchMapping testDNSToSwitchMapping =
+                new TestDNSToSwitchMapping(supMapRack0, supMapRack1, supMapRack2, supMapRack3, supMapRack4, supMapRack5);
+
+        //generate topologies
+        TopologyDetails topo1 = genTopology("topo-1", config, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
+        TopologyDetails topo2 = genTopology("topo-2", config, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
+
+        Topologies topologies = new Topologies(topo1, topo2);
+        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+
+        cluster.setNetworkTopography(testDNSToSwitchMapping.getRackToHosts());
+
+        NodeSorterHostProximity nodeSorter = new NodeSorterHostProximity(cluster, topo1);
+        nodeSorter.prepare(null);
+        List<ObjectResourcesItem> sortedRacks = StreamSupport.stream(nodeSorter.getSortedRacks().spliterator(), false)
+                .collect(Collectors.toList());
+        String rackSummaries = sortedRacks
+                .stream()
+                .map(x -> String.format("Rack %s -> scheduled-cnt %d, min-avail %f, avg-avail %f, cpu %f, mem %f",
+                        x.id, nodeSorter.getScheduledExecCntByRackId().getOrDefault(x.id, new AtomicInteger(-1)).get(),
+                        x.minResourcePercent, x.avgResourcePercent,
+                        x.availableResources.getTotalCpu(),
+                        x.availableResources.getTotalMemoryMb()))
+                .collect(Collectors.joining("\n\t"));
+        NormalizedResourceRequest topoResourceRequest = topo1.getApproximateTotalResources();
+        String topoRequest = String.format("Topo %s, approx-requested-resources %s", topo1.getId(), topoResourceRequest.toString());
+        Iterator<ObjectResourcesItem> it = sortedRacks.iterator();
+        assertEquals("rack-000", it.next().id, topoRequest + "\n\t" + rackSummaries + "\nRack-000 should be ordered first since it has the largest capacity");
+        assertEquals("rack-001", it.next().id, topoRequest + "\n\t" + rackSummaries + "\nrack-001 should be ordered second since it smaller than rack-000");
+        assertEquals("rack-002", it.next().id, topoRequest + "\n\t" + rackSummaries + "\nrack-002 should be ordered third since it is smaller than rack-001");
+        assertEquals("rack-003", it.next().id, topoRequest + "\n\t" + rackSummaries + "\nrack-003 should be ordered fourth since it since it is smaller than rack-002");
+        assertEquals("rack-004", it.next().id, topoRequest + "\n\t" + rackSummaries + "\nrack-004 should be ordered fifth since it since it is smaller than rack-003");
+        assertEquals("rack-005", it.next().id, topoRequest + "\n\t" + rackSummaries + "\nrack-005 should be ordered last since it since it is has smallest capacity");
+    }
+
+    /**
+     * Schedule two topologies, once with special resources and another without.
+     * There are enough special resources to hold one topology with special resource ("my.gpu").
+     * When using Round Robin scheduling, only one topology will be scheduled.
+     */
+    @Test
+    public void testAntiAffinityWithMultipleTopologies() {
+        INimbus iNimbus = new INimbusTest();
+        Map<String, SupervisorDetails> supMap = genSupervisorsWithRacks(1, 40, 66, 0, 0, 4700, 226200, new HashMap<>());
+        HashMap<String, Double> extraResources = new HashMap<>();
+        extraResources.put("my.gpu", 1.0);
+        supMap.putAll(genSupervisorsWithRacks(1, 40, 66, 1, 0, 4700, 226200, extraResources));
+
+        Config config = new Config();
+        config.putAll(createRoundRobinClusterConfig(88, 775, 25, null, null));
+
+        IScheduler scheduler = new ResourceAwareScheduler();
+        scheduler.prepare(config, new StormMetricsRegistry());
+
+        TopologyDetails tdSimple = genTopology("topology-simple", config, 1,
+                5, 100, 300, 0, 0, "user", 8192);
+
+        //Schedule the simple topology first
+        Topologies topologies = new Topologies(tdSimple);
+        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+
+        {
+            NodeSorterHostProximity nodeSorter = new NodeSorterHostProximity(cluster, tdSimple);
+            for (ExecutorDetails exec : tdSimple.getExecutors()) {
+                nodeSorter.prepare(exec);
+                List<ObjectResourcesItem> sortedRacks = StreamSupport
+                        .stream(nodeSorter.getSortedRacks().spliterator(), false)
+                        .collect(Collectors.toList());
+                String rackSummaries = StreamSupport
+                        .stream(sortedRacks.spliterator(), false)
+                        .map(x -> String.format("Rack %s -> scheduled-cnt %d, min-avail %f, avg-avail %f, cpu %f, mem %f",
+                                x.id, nodeSorter.getScheduledExecCntByRackId().getOrDefault(x.id, new AtomicInteger(-1)).get(),
+                                x.minResourcePercent, x.avgResourcePercent,
+                                x.availableResources.getTotalCpu(),
+                                x.availableResources.getTotalMemoryMb()))
+                        .collect(Collectors.joining("\n\t"));
+                NormalizedResourceRequest topoResourceRequest = tdSimple.getApproximateTotalResources();
+                String topoRequest = String.format("Topo %s, approx-requested-resources %s", tdSimple.getId(), topoResourceRequest.toString());
+                assertEquals(2, sortedRacks.size(), rackSummaries + "\n# of racks sorted");
+                assertEquals("rack-000", sortedRacks.get(0).id, rackSummaries + "\nFirst rack sorted");
+                assertEquals("rack-001", sortedRacks.get(1).id, rackSummaries + "\nSecond rack sorted");
+            }
+        }
+
+        scheduler.schedule(topologies, cluster);
+        Map<String, SchedulerAssignment> assignments = new TreeMap<>(cluster.getAssignments());
+        assertEquals(1, assignments.size());
+
+        TopologyBuilder builder = topologyBuilder(1, 5, 100, 300);
+        builder.setBolt("gpu-bolt", new TestBolt(), 40)
+                .addResource("my.gpu", 1.0)
+                .shuffleGrouping("spout-0");
+        TopologyDetails tdGpu = topoToTopologyDetails("topology-gpu", config, builder.createTopology(), 0, 0,"user", 8192);
+
+        //Now schedule GPU but with the simple topology in place.
+        topologies = new Topologies(tdSimple, tdGpu);
+        cluster = new Cluster(cluster, topologies);
+        {
+            NodeSorterHostProximity nodeSorter = new NodeSorterHostProximity(cluster, tdGpu);
+            for (ExecutorDetails exec : tdGpu.getExecutors()) {
+                String comp = tdGpu.getComponentFromExecutor(exec);
+                nodeSorter.prepare(exec);
+                List<ObjectResourcesItem> sortedRacks = StreamSupport
+                        .stream(nodeSorter.getSortedRacks().spliterator(), false).collect(Collectors.toList());
+                String rackSummaries = sortedRacks.stream()
+                        .map(x -> String.format("Rack %s -> scheduled-cnt %d, min-avail %f, avg-avail %f, cpu %f, mem %f",
+                                x.id, nodeSorter.getScheduledExecCntByRackId().getOrDefault(x.id, new AtomicInteger(-1)).get(),
+                                x.minResourcePercent, x.avgResourcePercent,
+                                x.availableResources.getTotalCpu(),
+                                x.availableResources.getTotalMemoryMb()))
+                        .collect(Collectors.joining("\n\t"));
+                NormalizedResourceRequest topoResourceRequest = tdSimple.getApproximateTotalResources();
+                String topoRequest = String.format("Topo %s, approx-requested-resources %s", tdSimple.getId(), topoResourceRequest.toString());
+                assertEquals(2, sortedRacks.size(), rackSummaries + "\n# of racks sorted");
+                if (comp.equals("gpu-bolt")) {
+                    assertEquals("rack-001", sortedRacks.get(0).id, rackSummaries + "\nFirst rack sorted for " + comp);
+                    assertEquals("rack-000", sortedRacks.get(1).id, rackSummaries + "\nSecond rack sorted for " + comp);
+                } else {
+                    assertEquals("rack-000", sortedRacks.get(0).id, rackSummaries + "\nFirst rack sorted for " + comp);
+                    assertEquals("rack-001", sortedRacks.get(1).id, rackSummaries + "\nSecond rack sorted for " + comp);
+                }
+            }
+        }
+
+        scheduler.schedule(topologies, cluster);
+
+        assignments = new TreeMap<>(cluster.getAssignments());
+        assertEquals(1, assignments.size()); // second topology is not expected to be assigned
+
+        Map<String, Map<String, AtomicLong>> topoPerRackCount = new HashMap<>();
+        for (Map.Entry<String, SchedulerAssignment> entry: assignments.entrySet()) {
+            SchedulerAssignment sa = entry.getValue();
+            Map<String, AtomicLong> slotsPerRack = new TreeMap<>();
+            for (WorkerSlot slot : sa.getSlots()) {
+                String nodeId = slot.getNodeId();
+                String rack = supervisorIdToRackName(nodeId);
+                slotsPerRack.computeIfAbsent(rack, (r) -> new AtomicLong(0)).incrementAndGet();
+            }
+            LOG.info("{} => {}", entry.getKey(), slotsPerRack);
+            topoPerRackCount.put(entry.getKey(), slotsPerRack);
+        }
+
+        Map<String, AtomicLong> simpleCount = topoPerRackCount.get("topology-simple-0");
+        assertNotNull(simpleCount);
+        //Because the simple topology was scheduled first we want to be sure that it didn't put anything on
+        // the GPU nodes.
+        assertEquals(2, simpleCount.size()); //Both racks are in use
+        assertTrue(simpleCount.containsKey("r001")); //r001 is the second rack with GPUs
+        assertTrue(simpleCount.containsKey("r000")); //r000 is the first rack with no GPUs
+
+        //We don't really care too much about the scheduling of topology-gpu-0, because it was scheduled.
+    }
+
+    /**
+     * Free one-fifth of WorkerSlots.
+     */
+    private void freeSomeWorkerSlots(Cluster cluster) {
+        Map<String, SchedulerAssignment> assignmentMap = cluster.getAssignments();
+        for (SchedulerAssignment schedulerAssignment: assignmentMap.values()) {
+            int i = 0;
+            List<WorkerSlot> slotsToKill = new ArrayList<>();
+            for (WorkerSlot workerSlot: schedulerAssignment.getSlots()) {
+                i++;
+                if (i % 5 == 0) {
+                    slotsToKill.add(workerSlot);
+                }
+            }
+            cluster.freeSlots(slotsToKill);
+        }
+    }
+
+    /**
+     * If the topology should be scheduled across all available racks instead of
+     * filling first rack and spilling on to the next rack.
+     */
+    @Test
+    public void testDistributeOverRacks() {
+        INimbus iNimbus = new INimbusTest();
+        double compPcore = 100;
+        double compOnHeap = 775;
+        double compOffHeap = 25;
+        int topo1NumSpouts = 1;
+        int topo1NumBolts = 5;
+        int topo1SpoutParallelism = 100;
+        int topo1BoltParallelism = 200;
+        final int numRacks = 3;
+        final int numSupersPerRack = 10;
+        final int numPortsPerSuper = 6;
+        final int numZonesPerHost = 1;
+        final double numaResourceMultiplier = 1.0;
+        int rackStartNum = 0;
+        int supStartNum = 0;
+        long compPerRack = (topo1NumSpouts * topo1SpoutParallelism + topo1NumBolts * topo1BoltParallelism) * 4/5; // not enough for topo1
+        long compPerSuper =  compPerRack / numSupersPerRack;
+        double cpuPerSuper = compPcore * compPerSuper;
+        double memPerSuper = (compOnHeap + compOffHeap) * compPerSuper;
+        double topo1MaxHeapSize = memPerSuper;
+        final String topoName1 = "topology1";
+
+        Map<String, SupervisorDetails> supMap = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum, supStartNum,
+                cpuPerSuper, memPerSuper, Collections.emptyMap(), numaResourceMultiplier);
+        TestDNSToSwitchMapping testDNSToSwitchMapping = new TestDNSToSwitchMapping(supMap.values());
+
+        Config config = new Config();
+        config.putAll(createRoundRobinClusterConfig(compPcore, compOnHeap, compOffHeap, null, null));
+
+        IScheduler scheduler = new ResourceAwareScheduler();
+        scheduler.prepare(config, new StormMetricsRegistry());
+
+        TopologyDetails td1 = genTopology(topoName1, config, topo1NumSpouts,
+                topo1NumBolts, topo1SpoutParallelism, topo1BoltParallelism, 0, 0, "user", topo1MaxHeapSize);
+
+        //Schedule the topo1 topology and ensure it fits on 2 racks
+        Topologies topologies = new Topologies(td1);
+        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+        cluster.setNetworkTopography(testDNSToSwitchMapping.getRackToHosts());
+
+        scheduler.schedule(topologies, cluster);
+        Set<String> assignedRacks = cluster.getAssignedRacks(td1.getId());
+        assertEquals(numRacks, assignedRacks.size(), "Racks for topology=" + td1.getId() + " is " + assignedRacks);
+    }
+
+    /**
+     * Racks are equally likely to be selected, rather than those with low resources already running components
+     * for the same topology.
+     * .
+     * <li>Schedule topo1 on one rack</li>
+     * <li>unassign some executors</li>
+     * <li>schedule another topology - cannot be scheduled since topo1 occupies all slots</li>
+     * <li>unassign topo2, kill workers and reschedule</li>
+     * <li>topo1 should utilize all all racks</li>
+     */
+    @Test
+    public void testDistributeAcrossRacks() {
+        INimbus iNimbus = new INimbusTest();
+        double compPcore = 100;
+        double compOnHeap = 775;
+        double compOffHeap = 25;
+        int topo1NumSpouts = 1;
+        int topo1NumBolts = 5;
+        int topo1SpoutParallelism = 100;
+        int topo1BoltParallelism = 200;
+        int topo2NumSpouts = 1;
+        int topo2NumBolts = 5;
+        int topo2SpoutParallelism = 10;
+        int topo2BoltParallelism = 20;
+        final int numRacks = 3;
+        final int numSupersPerRack = 10;
+        final int numPortsPerSuper = 6;
+        final int numZonesPerHost = 1;
+        final double numaResourceMultiplier = 1.0;
+        int rackStartNum = 0;
+        int supStartNum = 0;
+        long compPerRack = (topo1NumSpouts * topo1SpoutParallelism + topo1NumBolts * topo1BoltParallelism
+                + topo2NumSpouts * topo2SpoutParallelism); // enough for topo1 but not topo1+topo2
+        long compPerSuper =  compPerRack / numSupersPerRack;
+        double cpuPerSuper = compPcore * compPerSuper;
+        double memPerSuper = (compOnHeap + compOffHeap) * compPerSuper;
+        double topo1MaxHeapSize = memPerSuper;
+        double topo2MaxHeapSize = memPerSuper;
+        final String topoName1 = "topology1";
+        final String topoName2 = "topology2";
+
+        Map<String, SupervisorDetails> supMap = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum, supStartNum,
+                cpuPerSuper, memPerSuper, Collections.emptyMap(), numaResourceMultiplier);
+        TestDNSToSwitchMapping testDNSToSwitchMapping = new TestDNSToSwitchMapping(supMap.values());
+
+        Config config = new Config();
+        config.putAll(createRoundRobinClusterConfig(compPcore, compOnHeap, compOffHeap, null, null));
+
+        IScheduler scheduler = new ResourceAwareScheduler();
+        scheduler.prepare(config, new StormMetricsRegistry());
+
+        TopologyDetails td1 = genTopology(topoName1, config, topo1NumSpouts,
+                topo1NumBolts, topo1SpoutParallelism, topo1BoltParallelism, 0, 0, "user", topo1MaxHeapSize);
+
+        //Schedule the topo1 topology and ensure it fits on 1 rack
+        Topologies topologies = new Topologies(td1);
+        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+        cluster.setNetworkTopography(testDNSToSwitchMapping.getRackToHosts());
+
+        scheduler.schedule(topologies, cluster);
+        Set<String> assignedRacks = cluster.getAssignedRacks(td1.getId());
+        assertEquals(numRacks, assignedRacks.size(), "Racks for topology=" + td1.getId() + " is " + assignedRacks);
+
+        TopologyBuilder builder = topologyBuilder(topo2NumSpouts, topo2NumBolts, topo2SpoutParallelism, topo2BoltParallelism);
+        TopologyDetails td2 = topoToTopologyDetails(topoName2, config, builder.createTopology(), 0, 0,"user", topo2MaxHeapSize);
+
+        //Now schedule GPU but with the simple topology in place.
+        topologies = new Topologies(td1, td2);
+        cluster = new Cluster(cluster, topologies);
+        scheduler.schedule(topologies, cluster);
+
+        assignedRacks = cluster.getAssignedRacks(td1.getId(), td2.getId());
+        assertEquals(numRacks, assignedRacks.size(), "Racks for topologies=" + td1.getId() + "/" + td2.getId() + " is " + assignedRacks);
+
+        // topo2 will not get scheduled as topo1 will occupy all racks
+        assignedRacks = cluster.getAssignedRacks(td2.getId());
+        assertEquals(0, assignedRacks.size(), "Racks for topologies=" + td2.getId() + " is " + assignedRacks);
+
+        // now unassign topo2, expect all racks to be in use; free some slots and reschedule topo1 some topo1 executors
+        cluster.unassign(td2.getId());
+        assignedRacks = cluster.getAssignedRacks(td2.getId());
+        assertEquals(0, assignedRacks.size(),
+            "After unassigning topology " + td2.getId() + ", racks for topology=" + td2.getId() + " is " + assignedRacks);
+        assignedRacks = cluster.getAssignedRacks(td1.getId());
+        assertEquals(numRacks, assignedRacks.size(),
+            "After unassigning topology " + td2.getId() + ", racks for topology=" + td1.getId() + " is " + assignedRacks);
+        assertFalse(cluster.needsSchedulingRas(td1),
+            "Topology " + td1.getId() + " should be fully assigned before freeing slots");
+        freeSomeWorkerSlots(cluster);
+        assertTrue(cluster.needsSchedulingRas(td1),
+            "Topology " + td1.getId() + " should need scheduling after freeing slots");
+
+        // then reschedule executors
+        scheduler.schedule(topologies, cluster);
+
+        // all racks should be in use by topology1
+        assignedRacks = cluster.getAssignedRacks(td1.getId());
+        assertEquals(numRacks, assignedRacks.size(),
+            "After reassigning topology " + td2.getId() + ", racks for topology=" + td1.getId() + " is " + assignedRacks);
+    }
+
+    /**
+     * Assign and then clear out a rack to host list mapping in cluster.networkTopography.
+     * Expected behavior is that:
+     *  <li>the rack without hosts does not show up in {@link NodeSorterHostProximity#getSortedRacks()}</li>
+     *  <li>all the supervisor nodes still get returned in {@link NodeSorterHostProximity#sortAllNodes()} ()}</li>
+     *  <li>supervisors on cleared rack show up under {@link DNSToSwitchMapping#DEFAULT_RACK}</li>
+     *
+     *  <p>
+     *      Force an usual condition, where one of the racks is still passed to LazyNodeSortingIterator with
+     *      an empty list and then ensure that code is resilient.
+     *  </p>
+     */
+    @Test
+    void testWithImpairedClusterNetworkTopography() {
+        INimbus iNimbus = new INimbusTest();
+        double compPcore = 100;
+        double compOnHeap = 775;
+        double compOffHeap = 25;
+        int topo1NumSpouts = 1;
+        int topo1NumBolts = 5;
+        int topo1SpoutParallelism = 100;
+        int topo1BoltParallelism = 200;
+        final int numSupersPerRack = 10;
+        final int numPortsPerSuper = 66;
+        long compPerRack = (topo1NumSpouts * topo1SpoutParallelism + topo1NumBolts * topo1BoltParallelism + 10);
+        long compPerSuper =  compPerRack / numSupersPerRack;
+        double cpuPerSuper = compPcore * compPerSuper;
+        double memPerSuper = (compOnHeap + compOffHeap) * compPerSuper;
+        double topo1MaxHeapSize = memPerSuper;
+        final String topoName1 = "topology1";
+        int numRacks = 3;
+
+        Map<String, SupervisorDetails> supMap = genSupervisorsWithRacks(numRacks, numSupersPerRack,  numPortsPerSuper,
+            0, 0, cpuPerSuper, memPerSuper, new HashMap<>());
+        TestDNSToSwitchMapping testDNSToSwitchMapping = new TestDNSToSwitchMapping(supMap.values());
+
+        Config config = new Config();
+        config.putAll(createRoundRobinClusterConfig(compPcore, compOnHeap, compOffHeap, null, null));
+
+        IScheduler scheduler = new ResourceAwareScheduler();
+        scheduler.prepare(config, new StormMetricsRegistry());
+
+        TopologyDetails td1 = genTopology(topoName1, config, topo1NumSpouts,
+            topo1NumBolts, topo1SpoutParallelism, topo1BoltParallelism, 0, 0, "user", topo1MaxHeapSize);
+
+        Topologies topologies = new Topologies(td1);
+        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+        cluster.setNetworkTopography(testDNSToSwitchMapping.getRackToHosts());
+
+        Map<String, List<String>> networkTopography = cluster.getNetworkTopography();
+        assertEquals(numRacks, networkTopography.size(), "Expecting " + numRacks + " racks found " + networkTopography.size());
+        assertTrue(networkTopography.size() >= 3, "Expecting racks count to be >= 3, found " + networkTopography.size());
+
+        // Impair cluster.networkTopography and set one rack to have zero hosts, getSortedRacks should exclude this rack.
+        // Keep, the supervisorDetails unchanged - confirm that these nodes are not lost even with incomplete networkTopography
+        String rackIdToZero = networkTopography.keySet().stream().findFirst().get();
+        impairClusterRack(cluster, rackIdToZero, true, false);
+
+        NodeSorterHostProximity nodeSorterHostProximity = new NodeSorterHostProximity(cluster, td1);
+        nodeSorterHostProximity.getSortedRacks().forEach(x -> assertNotEquals(x.id, rackIdToZero));
+
+        // confirm that the above action has not lost the hosts and that they appear under the DEFAULT rack
+        {
+            Set<String> seenRacks = new HashSet<>();
+            nodeSorterHostProximity.getSortedRacks().forEach(x -> seenRacks.add(x.id));
+            assertEquals(numRacks, seenRacks.size(), "Expecting rack cnt to be still " + numRacks);
+            assertTrue(seenRacks.contains(DNSToSwitchMapping.DEFAULT_RACK),
+                "Expecting to see default-rack=" + DNSToSwitchMapping.DEFAULT_RACK + " in sortedRacks");
+        }
+
+        // now check if node/supervisor is missing when sorting all nodes
+        Set<String> expectedNodes = supMap.keySet();
+        Set<String> seenNodes = new HashSet<>();
+        nodeSorterHostProximity.prepare(null);
+        nodeSorterHostProximity.sortAllNodes().forEach( n -> seenNodes.add(n));
+        assertEquals(expectedNodes, seenNodes, "Expecting see all supervisors ");
+
+        // Now fully impair the cluster - confirm no default rack
+        {
+            cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+            cluster.setNetworkTopography(new TestDNSToSwitchMapping(supMap.values()).getRackToHosts());
+            impairClusterRack(cluster, rackIdToZero, true, true);
+            Set<String> seenRacks = new HashSet<>();
+            NodeSorterHostProximity nodeSorterHostProximity2 = new NodeSorterHostProximity(cluster, td1);
+            nodeSorterHostProximity2.getSortedRacks().forEach(x -> seenRacks.add(x.id));
+            Map<String, Set<String>> rackIdToHosts = nodeSorterHostProximity2.getRackIdToHosts();
+            String dumpOfRacks = rackIdToHosts.entrySet().stream()
+                .map(x -> String.format("rack %s -> hosts [%s]", x.getKey(), String.join(",", x.getValue())))
+                .collect(Collectors.joining("\n\t"));
+            assertEquals(numRacks - 1, seenRacks.size(),
+                "Expecting rack cnt to be " + (numRacks - 1) + " but found " + seenRacks.size() + "\n\t" + dumpOfRacks);
+            assertFalse(seenRacks.contains(DNSToSwitchMapping.DEFAULT_RACK),
+                "Found default-rack=" + DNSToSwitchMapping.DEFAULT_RACK + " in \n\t" + dumpOfRacks);
+        }
+    }
+
+    /**
+     * Black list all nodes for a rack before sorting nodes.
+     * Confirm that {@link NodeSorterHostProximity#sortAllNodes()} still works.
+     *
+     */
+    @Test
+    void testWithBlackListedHosts() {
+        INimbus iNimbus = new INimbusTest();
+        double compPcore = 100;
+        double compOnHeap = 775;
+        double compOffHeap = 25;
+        int topo1NumSpouts = 1;
+        int topo1NumBolts = 5;
+        int topo1SpoutParallelism = 100;
+        int topo1BoltParallelism = 200;
+        final int numSupersPerRack = 10;
+        final int numPortsPerSuper = 66;
+        long compPerRack = (topo1NumSpouts * topo1SpoutParallelism + topo1NumBolts * topo1BoltParallelism + 10);
+        long compPerSuper =  compPerRack / numSupersPerRack;
+        double cpuPerSuper = compPcore * compPerSuper;
+        double memPerSuper = (compOnHeap + compOffHeap) * compPerSuper;
+        double topo1MaxHeapSize = memPerSuper;
+        final String topoName1 = "topology1";
+        int numRacks = 3;
+
+        Map<String, SupervisorDetails> supMap = genSupervisorsWithRacks(numRacks, numSupersPerRack,  numPortsPerSuper,
+            0, 0, cpuPerSuper, memPerSuper, new HashMap<>());
+        TestDNSToSwitchMapping testDNSToSwitchMapping = new TestDNSToSwitchMapping(supMap.values());
+
+        Config config = new Config();
+        config.putAll(createRoundRobinClusterConfig(compPcore, compOnHeap, compOffHeap, null, null));
+
+        IScheduler scheduler = new ResourceAwareScheduler();
+        scheduler.prepare(config, new StormMetricsRegistry());
+
+        TopologyDetails td1 = genTopology(topoName1, config, topo1NumSpouts,
+            topo1NumBolts, topo1SpoutParallelism, topo1BoltParallelism, 0, 0, "user", topo1MaxHeapSize);
+
+        Topologies topologies = new Topologies(td1);
+        Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
+        cluster.setNetworkTopography(testDNSToSwitchMapping.getRackToHosts());
+
+        Map<String, List<String>> networkTopography = cluster.getNetworkTopography();
+        assertEquals(numRacks, networkTopography.size(), "Expecting " + numRacks + " racks found " + networkTopography.size());
+        assertTrue(networkTopography.size() >= 3, "Expecting racks count to be >= 3, found " + networkTopography.size());
+
+        Set<String> blackListedHosts = new HashSet<>();
+        List<SupervisorDetails> supArray = new ArrayList<>(supMap.values());
+        for (int i = 0 ; i < numSupersPerRack ; i++) {
+            blackListedHosts.add(supArray.get(i).getHost());
+        }
+        blacklistHostsAndSortNodes(blackListedHosts, supMap.values(), cluster, td1);
+
+        String rackToClear = cluster.getNetworkTopography().keySet().stream().findFirst().get();
+        blackListedHosts = new HashSet<>(cluster.getNetworkTopography().get(rackToClear));
+        blacklistHostsAndSortNodes(blackListedHosts, supMap.values(), cluster, td1);
+    }
+
+    // Impair cluster by blacklisting some hosts
+    private void blacklistHostsAndSortNodes(
+        Set<String> blackListedHosts, Collection<SupervisorDetails> sups, Cluster cluster, TopologyDetails td1) {
+        LOG.info("blackListedHosts={}", blackListedHosts);
+        cluster.setBlacklistedHosts(blackListedHosts);
+
+        NodeSorterHostProximity nodeSorterHostProximity = new NodeSorterHostProximity(cluster, td1);
+        // confirm that the above action loses hosts
+        {
+            Set<String> allHosts = sups.stream().map(x -> x.getHost()).collect(Collectors.toSet());
+            Set<String> seenRacks = new HashSet<>();
+            nodeSorterHostProximity.getSortedRacks().forEach(x -> seenRacks.add(x.id));
+            Set<String> seenHosts = new HashSet<>();
+            nodeSorterHostProximity.getRackIdToHosts().forEach((k,v) -> seenHosts.addAll(v));
+            allHosts.removeAll(seenHosts);
+            assertEquals(allHosts, blackListedHosts, "Expecting only blacklisted hosts removed");
+        }
+
+        // now check if sortAllNodes still works
+        Set<String> expectedNodes = sups.stream()
+            .filter(x -> !blackListedHosts.contains(x.getHost()))
+            .map(x ->x.getId())
+            .collect(Collectors.toSet());
+        Set<String> seenNodes = new HashSet<>();
+            nodeSorterHostProximity.prepare(null);
+            nodeSorterHostProximity.sortAllNodes().forEach( n -> seenNodes.add(n));
+        assertEquals(expectedNodes, seenNodes, "Expecting see all supervisors ");
+    }
+
+    /**
+     * Impair the cluster for a specified rackId.
+     *  <li>making the host list a zero length</li>
+     *  <li>removing supervisors for the hosts on the rack</li>
+     *
+     * @param cluster cluster to impair
+     * @param rackId rackId to clear
+     * @param clearNetworkTopography if true, then clear (but not remove) the hosts in list for the rack.
+     * @param clearSupervisorMap if true, then remove supervisors for the rack.
+     */
+    private void impairClusterRack(Cluster cluster, String rackId, boolean clearNetworkTopography, boolean clearSupervisorMap) {
+        Set<String> hostIds = new HashSet<>(cluster.getNetworkTopography().computeIfAbsent(rackId, k -> new ArrayList<>()));
+        if (clearNetworkTopography) {
+            cluster.getNetworkTopography().computeIfAbsent(rackId, k -> new ArrayList<>()).clear();
+        }
+        if (clearSupervisorMap) {
+            Set<String> supToRemove = new HashSet<>();
+            for (String hostId: hostIds) {
+                cluster.getSupervisorsByHost(hostId).forEach(s -> supToRemove.add(s.getId()));
+            }
+            Map<String, SupervisorDetails> supervisorDetailsMap = cluster.getSupervisors();
+            for (String supId: supToRemove) {
+                supervisorDetailsMap.remove(supId);
+            }
+        }
+    }
+}
\ No newline at end of file