You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by GitBox <gi...@apache.org> on 2021/03/09 22:52:40 UTC

[GitHub] [storm] Ethanlm commented on a change in pull request #3379: [STORM-3739] Scheduling should sort numa zones by host groups

Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r590746755



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,712 @@
+/*
+ * 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.sorter;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.storm.Config;
+import org.apache.storm.networktopography.DNSToSwitchMapping;
+import org.apache.storm.scheduler.Cluster;
+import org.apache.storm.scheduler.ExecutorDetails;
+import org.apache.storm.scheduler.SchedulerAssignment;
+import org.apache.storm.scheduler.TopologyDetails;
+import org.apache.storm.scheduler.WorkerSlot;
+import org.apache.storm.scheduler.resource.RasNode;
+import org.apache.storm.scheduler.resource.RasNodes;
+import org.apache.storm.scheduler.resource.normalization.NormalizedResourceOffer;
+import org.apache.storm.scheduler.resource.normalization.NormalizedResourceRequest;
+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.ObjectResourcesSummary;
+import org.apache.storm.shade.com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NodeSorterHostProximity implements INodeSorter {
+    private static final Logger LOG = LoggerFactory.getLogger(NodeSorterHostProximity.class);
+
+    // instance variables from class instantiation
+    protected final BaseResourceAwareStrategy.NodeSortType nodeSortType;
+
+    protected Cluster cluster;
+    protected TopologyDetails topologyDetails;
+
+    // Instance variables derived from Cluster.
+    private final Map<String, List<String>> networkTopography;

Review comment:
       This doesn't seem to be used anywhere. It can be removed.

##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,712 @@
+/*
+ * 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.sorter;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.storm.Config;
+import org.apache.storm.networktopography.DNSToSwitchMapping;
+import org.apache.storm.scheduler.Cluster;
+import org.apache.storm.scheduler.ExecutorDetails;
+import org.apache.storm.scheduler.SchedulerAssignment;
+import org.apache.storm.scheduler.TopologyDetails;
+import org.apache.storm.scheduler.WorkerSlot;
+import org.apache.storm.scheduler.resource.RasNode;
+import org.apache.storm.scheduler.resource.RasNodes;
+import org.apache.storm.scheduler.resource.normalization.NormalizedResourceOffer;
+import org.apache.storm.scheduler.resource.normalization.NormalizedResourceRequest;
+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.ObjectResourcesSummary;
+import org.apache.storm.shade.com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NodeSorterHostProximity implements INodeSorter {
+    private static final Logger LOG = LoggerFactory.getLogger(NodeSorterHostProximity.class);
+
+    // instance variables from class instantiation
+    protected final BaseResourceAwareStrategy.NodeSortType nodeSortType;
+
+    protected Cluster cluster;
+    protected TopologyDetails topologyDetails;
+
+    // Instance variables derived from Cluster.
+    private final Map<String, List<String>> networkTopography;
+    private final Map<String, String> superIdToRack = new HashMap<>();
+    private final Map<String, List<RasNode>> hostnameToNodes = new HashMap<>();
+    private final Map<String, String> nodeIdToHostname = new HashMap<>();
+    private final Map<String, Set<String>> rackIdToHosts = new HashMap<>();
+    protected List<String> greyListedSupervisorIds;
+
+    // Instance variables from Cluster and TopologyDetails.
+    protected List<String> favoredNodeIds;
+    protected List<String> unFavoredNodeIds;
+
+    // Updated in prepare method
+    ExecutorDetails exec;
+
+    public NodeSorterHostProximity(Cluster cluster, TopologyDetails topologyDetails) {
+        this(cluster, topologyDetails, BaseResourceAwareStrategy.NodeSortType.COMMON);
+    }
+
+    /**
+     * Initialize for the default implementation node sorting.
+     *
+     * <p>
+     *  <li>{@link BaseResourceAwareStrategy.NodeSortType#GENERIC_RAS} sorting implemented in
+     *  {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, NodeSorterHostProximity.ExistingScheduleFunc)}</li>
+     *  <li>{@link BaseResourceAwareStrategy.NodeSortType#DEFAULT_RAS} sorting implemented in
+     *  {@link #sortObjectResourcesDefault(ObjectResourcesSummary, NodeSorterHostProximity.ExistingScheduleFunc)}</li>
+     *  <li>{@link BaseResourceAwareStrategy.NodeSortType#COMMON} sorting implemented in
+     *  {@link #sortObjectResourcesCommon(ObjectResourcesSummary, ExecutorDetails, NodeSorterHostProximity.ExistingScheduleFunc)}</li>
+     * </p>
+     *
+     * @param cluster for which nodes will be sorted.
+     * @param topologyDetails the topology to sort for.
+     * @param nodeSortType type of sorting to be applied to object resource collection {@link BaseResourceAwareStrategy.NodeSortType}.
+     */
+    public NodeSorterHostProximity(Cluster cluster, TopologyDetails topologyDetails, BaseResourceAwareStrategy.NodeSortType nodeSortType) {
+        this.cluster = cluster;
+        this.topologyDetails = topologyDetails;
+        this.nodeSortType = nodeSortType;
+
+        // from Cluster
+        networkTopography = cluster.getNetworkTopography();
+        greyListedSupervisorIds = cluster.getGreyListedSupervisors();
+        Map<String, String> hostToRack = cluster.getHostToRack();
+        RasNodes nodes = new RasNodes(cluster);
+        for (RasNode node: nodes.getNodes()) {
+            String superId = node.getId();
+            String hostName = node.getHostname();
+            if (!node.isAlive() || hostName == null) {
+                continue;
+            }
+            String rackId = hostToRack.getOrDefault(hostName, DNSToSwitchMapping.DEFAULT_RACK);
+            superIdToRack.put(superId, rackId);
+            hostnameToNodes.computeIfAbsent(hostName, (hn) -> new ArrayList<>()).add(node);
+            nodeIdToHostname.put(superId, hostName);
+            rackIdToHosts.computeIfAbsent(rackId, r -> new HashSet<>()).add(hostName);
+        }
+
+        // from TopologyDetails
+        Map<String, Object> topoConf = topologyDetails.getConf();
+
+        // From Cluster and TopologyDetails - and cleaned-up
+        favoredNodeIds = makeHostToNodeIds((List<String>) topoConf.get(Config.TOPOLOGY_SCHEDULER_FAVORED_NODES));
+        unFavoredNodeIds = makeHostToNodeIds((List<String>) topoConf.get(Config.TOPOLOGY_SCHEDULER_UNFAVORED_NODES));
+        favoredNodeIds.removeAll(greyListedSupervisorIds);
+        unFavoredNodeIds.removeAll(greyListedSupervisorIds);
+        unFavoredNodeIds.removeAll(favoredNodeIds);
+    }
+
+    @VisibleForTesting
+    public Map<String, Set<String>> getRackIdToHosts() {
+        return rackIdToHosts;
+    }
+
+    @Override
+    public void prepare(ExecutorDetails exec) {
+        this.exec = exec;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.

Review comment:
       nit : can be altered

##########
File path: storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/TestNodeSorterHostProximity.java
##########
@@ -0,0 +1,1054 @@
+/*
+ * 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.logging.log4j.Level;
+import org.apache.logging.log4j.core.config.Configurator;
+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.DefaultResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.ObjectResourcesItem;
+import org.apache.storm.topology.TopologyBuilder;
+import org.junit.Assert;
+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.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+@ExtendWith({NormalizedResourcesExtension.class})
+public class TestNodeSorterHostProximity {
+    private static final Logger LOG = LoggerFactory.getLogger(TestNodeSorterHostProximity.class);
+    private static final int CURRENT_TIME = 1450418597;
+
+    protected Class getDefaultResourceAwareStrategyClass() {
+        return DefaultResourceAwareStrategy.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, getDefaultResourceAwareStrategyClass().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 testMultipleRacks() {
+        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,
+                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);
+
+        //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
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack5 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                0.0, 0.0, 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);
+
+        List<String> supHostnames = new LinkedList<>();
+        for (SupervisorDetails sup : supMap.values()) {
+            supHostnames.add(sup.getHost());
+        }
+        Map<String, List<String>> rackToHosts = testDNSToSwitchMapping.getRackToHosts();
+        cluster.setNetworkTopography(rackToHosts);
+        System.out.println("DEBUG: rackToHosts: " + rackToHosts.toString());
+
+        NodeSorterHostProximity nodeSorter = new NodeSorterHostProximity(cluster, topo1, BaseResourceAwareStrategy.NodeSortType.DEFAULT_RAS);
+        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"));
+        Assert.assertEquals(rackSummaries + "\n# of racks sorted", 6, sortedRacks.size());
+        Iterator<ObjectResourcesItem> it = sortedRacks.iterator();
+        Assert.assertEquals(rackSummaries + "\nrack-000 should be ordered first since it has the most balanced set of resources", "rack-000", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-001 should be ordered second since it has a balanced set of resources but less than rack-000", "rack-001", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-004 should be ordered third since it has a lot of cpu but not a lot of memory", "rack-004", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-003 should be ordered fourth since it has a lot of memory but not cpu", "rack-003", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-002 should be ordered fifth since it has not cpu resources", "rack-002", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nRack-005 should be ordered sixth since it has neither CPU nor memory available", "rack-005", it.next().id);
+    }
+
+    /**
+     * 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.DEFAULT_RAS);
+        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
+        Assert.assertEquals("rack-000 should be ordered first", "rack-000", it.next().id);
+        // Ranked second since rack-1 has a balanced set of resources but less than rack-0
+        Assert.assertEquals("rack-001 should be ordered second", "rack-001", it.next().id);
+        // Ranked third since rack-4 has a lot of cpu but not a lot of memory
+        Assert.assertEquals("rack-004 should be ordered third", "rack-004", it.next().id);
+        // Ranked fourth since rack-3 has alot of memory but not cpu
+        Assert.assertEquals("rack-003 should be ordered fourth", "rack-003", it.next().id);
+        //Ranked last since rack-2 has not cpu resources
+        Assert.assertEquals("rack-00s2 should be ordered fifth", "rack-002", it.next().id);
+    }
+
+    /**
+     * 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));
+                Assert.fail(err);
+            }
+            seenHosts.add(host);
+            prevHost = host;
+        }
+        System.out.printf("DEBUG: Test Success: Hosts are listed together:\n\t%s\n", String.join("\n\t", errLines));
+    }
+
+    /**
+     * 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());
+        //Assert.assertEquals(rackSummaries + "\n# of racks sorted", 6, sortedRacks.size());

Review comment:
       Remove unused code; or uncomment it if it is used. 
   Looks like we should keep it.

##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorter.java
##########
@@ -122,16 +122,21 @@ public NodeSorter(Cluster cluster, TopologyDetails topologyDetails, BaseResource
         unFavoredNodeIds.removeAll(favoredNodeIds);
     }
 
+    @Override
+    public void prepare(ExecutorDetails exec) {
+        this.exec = exec;
+    }
+
     /**
-     * Scheduling uses {@link #sortAllNodes(ExecutorDetails)} which eventually
+     * Scheduling uses {@link #sortAllNodes()} which eventually
      * calls this method whose behavior can altered by setting {@link #nodeSortType}.

Review comment:
       nit : can be altered

##########
File path: storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/TestNodeSorterHostProximity.java
##########
@@ -0,0 +1,1054 @@
+/*
+ * 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.logging.log4j.Level;
+import org.apache.logging.log4j.core.config.Configurator;
+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.DefaultResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.ObjectResourcesItem;
+import org.apache.storm.topology.TopologyBuilder;
+import org.junit.Assert;
+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.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+@ExtendWith({NormalizedResourcesExtension.class})
+public class TestNodeSorterHostProximity {
+    private static final Logger LOG = LoggerFactory.getLogger(TestNodeSorterHostProximity.class);
+    private static final int CURRENT_TIME = 1450418597;
+
+    protected Class getDefaultResourceAwareStrategyClass() {
+        return DefaultResourceAwareStrategy.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, getDefaultResourceAwareStrategyClass().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 testMultipleRacks() {
+        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,
+                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);
+
+        //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
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack5 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                0.0, 0.0, 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);
+
+        List<String> supHostnames = new LinkedList<>();
+        for (SupervisorDetails sup : supMap.values()) {
+            supHostnames.add(sup.getHost());
+        }
+        Map<String, List<String>> rackToHosts = testDNSToSwitchMapping.getRackToHosts();
+        cluster.setNetworkTopography(rackToHosts);
+        System.out.println("DEBUG: rackToHosts: " + rackToHosts.toString());
+
+        NodeSorterHostProximity nodeSorter = new NodeSorterHostProximity(cluster, topo1, BaseResourceAwareStrategy.NodeSortType.DEFAULT_RAS);
+        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"));
+        Assert.assertEquals(rackSummaries + "\n# of racks sorted", 6, sortedRacks.size());
+        Iterator<ObjectResourcesItem> it = sortedRacks.iterator();
+        Assert.assertEquals(rackSummaries + "\nrack-000 should be ordered first since it has the most balanced set of resources", "rack-000", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-001 should be ordered second since it has a balanced set of resources but less than rack-000", "rack-001", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-004 should be ordered third since it has a lot of cpu but not a lot of memory", "rack-004", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-003 should be ordered fourth since it has a lot of memory but not cpu", "rack-003", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-002 should be ordered fifth since it has not cpu resources", "rack-002", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nRack-005 should be ordered sixth since it has neither CPU nor memory available", "rack-005", it.next().id);
+    }
+
+    /**
+     * 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.DEFAULT_RAS);
+        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
+        Assert.assertEquals("rack-000 should be ordered first", "rack-000", it.next().id);
+        // Ranked second since rack-1 has a balanced set of resources but less than rack-0
+        Assert.assertEquals("rack-001 should be ordered second", "rack-001", it.next().id);
+        // Ranked third since rack-4 has a lot of cpu but not a lot of memory
+        Assert.assertEquals("rack-004 should be ordered third", "rack-004", it.next().id);
+        // Ranked fourth since rack-3 has alot of memory but not cpu
+        Assert.assertEquals("rack-003 should be ordered fourth", "rack-003", it.next().id);
+        //Ranked last since rack-2 has not cpu resources
+        Assert.assertEquals("rack-00s2 should be ordered fifth", "rack-002", it.next().id);
+    }
+
+    /**
+     * 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));
+                Assert.fail(err);
+            }
+            seenHosts.add(host);
+            prevHost = host;
+        }
+        System.out.printf("DEBUG: Test Success: Hosts are listed together:\n\t%s\n", String.join("\n\t", errLines));
+    }
+
+    /**
+     * 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());
+        //Assert.assertEquals(rackSummaries + "\n# of racks sorted", 6, sortedRacks.size());
+        Iterator<ObjectResourcesItem> it = sortedRacks.iterator();
+        Assert.assertEquals(topoRequest + "\n\t" + rackSummaries + "\nRack-000 should be ordered first since it has the largest capacity", "rack-000", it.next().id);
+        Assert.assertEquals(topoRequest + "\n\t" + rackSummaries + "\nrack-001 should be ordered second since it smaller than rack-000", "rack-001", it.next().id);
+        Assert.assertEquals(topoRequest + "\n\t" + rackSummaries + "\nrack-002 should be ordered third since it is smaller than rack-001", "rack-002", it.next().id);
+        Assert.assertEquals(topoRequest + "\n\t" + rackSummaries + "\nrack-003 should be ordered fourth since it since it is smaller than rack-002", "rack-003", it.next().id);
+        Assert.assertEquals(topoRequest + "\n\t" + rackSummaries + "\nrack-004 should be ordered fifth since it since it is smaller than rack-003", "rack-004", it.next().id);
+        Assert.assertEquals(topoRequest + "\n\t" + rackSummaries + "\nrack-005 should be ordered last since it since it is has smallest capacity", "rack-005", it.next().id);
+    }
+
+    /**
+     * Schedule two topologies, once with special resources and another without.
+     * There are enough special resources to hold one topology with special resource ("my.gpu").
+     * If the sort order is incorrect, scheduling will not succeed.
+     */
+    @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));
+        //config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, GenericResourceAwareStrategy.class.getName());

Review comment:
       Remove unused code; or uncomment it if it is used

##########
File path: storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/TestNodeSorterHostProximity.java
##########
@@ -0,0 +1,1054 @@
+/*
+ * 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.logging.log4j.Level;
+import org.apache.logging.log4j.core.config.Configurator;
+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.DefaultResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.ObjectResourcesItem;
+import org.apache.storm.topology.TopologyBuilder;
+import org.junit.Assert;
+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.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+@ExtendWith({NormalizedResourcesExtension.class})
+public class TestNodeSorterHostProximity {
+    private static final Logger LOG = LoggerFactory.getLogger(TestNodeSorterHostProximity.class);
+    private static final int CURRENT_TIME = 1450418597;
+
+    protected Class getDefaultResourceAwareStrategyClass() {
+        return DefaultResourceAwareStrategy.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, getDefaultResourceAwareStrategyClass().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 testMultipleRacks() {
+        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,
+                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);
+
+        //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
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack5 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                0.0, 0.0, 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);
+
+        List<String> supHostnames = new LinkedList<>();
+        for (SupervisorDetails sup : supMap.values()) {
+            supHostnames.add(sup.getHost());
+        }
+        Map<String, List<String>> rackToHosts = testDNSToSwitchMapping.getRackToHosts();
+        cluster.setNetworkTopography(rackToHosts);
+        System.out.println("DEBUG: rackToHosts: " + rackToHosts.toString());

Review comment:
       Should we use `LOG.info` here?

##########
File path: storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/TestNodeSorterHostProximity.java
##########
@@ -0,0 +1,1054 @@
+/*
+ * 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.logging.log4j.Level;
+import org.apache.logging.log4j.core.config.Configurator;
+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.DefaultResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.ObjectResourcesItem;
+import org.apache.storm.topology.TopologyBuilder;
+import org.junit.Assert;
+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.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+@ExtendWith({NormalizedResourcesExtension.class})
+public class TestNodeSorterHostProximity {
+    private static final Logger LOG = LoggerFactory.getLogger(TestNodeSorterHostProximity.class);
+    private static final int CURRENT_TIME = 1450418597;
+
+    protected Class getDefaultResourceAwareStrategyClass() {
+        return DefaultResourceAwareStrategy.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, getDefaultResourceAwareStrategyClass().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 testMultipleRacks() {
+        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,
+                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);
+
+        //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
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack5 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                0.0, 0.0, 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);
+
+        List<String> supHostnames = new LinkedList<>();
+        for (SupervisorDetails sup : supMap.values()) {
+            supHostnames.add(sup.getHost());
+        }
+        Map<String, List<String>> rackToHosts = testDNSToSwitchMapping.getRackToHosts();
+        cluster.setNetworkTopography(rackToHosts);
+        System.out.println("DEBUG: rackToHosts: " + rackToHosts.toString());
+
+        NodeSorterHostProximity nodeSorter = new NodeSorterHostProximity(cluster, topo1, BaseResourceAwareStrategy.NodeSortType.DEFAULT_RAS);
+        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"));
+        Assert.assertEquals(rackSummaries + "\n# of racks sorted", 6, sortedRacks.size());
+        Iterator<ObjectResourcesItem> it = sortedRacks.iterator();
+        Assert.assertEquals(rackSummaries + "\nrack-000 should be ordered first since it has the most balanced set of resources", "rack-000", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-001 should be ordered second since it has a balanced set of resources but less than rack-000", "rack-001", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-004 should be ordered third since it has a lot of cpu but not a lot of memory", "rack-004", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-003 should be ordered fourth since it has a lot of memory but not cpu", "rack-003", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-002 should be ordered fifth since it has not cpu resources", "rack-002", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nRack-005 should be ordered sixth since it has neither CPU nor memory available", "rack-005", it.next().id);
+    }
+
+    /**
+     * 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.DEFAULT_RAS);
+        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
+        Assert.assertEquals("rack-000 should be ordered first", "rack-000", it.next().id);
+        // Ranked second since rack-1 has a balanced set of resources but less than rack-0
+        Assert.assertEquals("rack-001 should be ordered second", "rack-001", it.next().id);
+        // Ranked third since rack-4 has a lot of cpu but not a lot of memory
+        Assert.assertEquals("rack-004 should be ordered third", "rack-004", it.next().id);
+        // Ranked fourth since rack-3 has alot of memory but not cpu
+        Assert.assertEquals("rack-003 should be ordered fourth", "rack-003", it.next().id);
+        //Ranked last since rack-2 has not cpu resources
+        Assert.assertEquals("rack-00s2 should be ordered fifth", "rack-002", it.next().id);
+    }
+
+    /**
+     * 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));
+                Assert.fail(err);
+            }
+            seenHosts.add(host);
+            prevHost = host;
+        }
+        System.out.printf("DEBUG: Test Success: Hosts are listed together:\n\t%s\n", String.join("\n\t", errLines));

Review comment:
       Should we use `LOG.info` here?

##########
File path: storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/TestNodeSorterHostProximity.java
##########
@@ -0,0 +1,1054 @@
+/*
+ * 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.logging.log4j.Level;
+import org.apache.logging.log4j.core.config.Configurator;
+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.DefaultResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategy;
+import org.apache.storm.scheduler.resource.strategies.scheduling.ObjectResourcesItem;
+import org.apache.storm.topology.TopologyBuilder;
+import org.junit.Assert;
+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.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+@ExtendWith({NormalizedResourcesExtension.class})
+public class TestNodeSorterHostProximity {
+    private static final Logger LOG = LoggerFactory.getLogger(TestNodeSorterHostProximity.class);
+    private static final int CURRENT_TIME = 1450418597;
+
+    protected Class getDefaultResourceAwareStrategyClass() {
+        return DefaultResourceAwareStrategy.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, getDefaultResourceAwareStrategyClass().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 testMultipleRacks() {
+        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,
+                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);
+
+        //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
+        supStartNum += numSupersPerRack;
+        final Map<String, SupervisorDetails> supMapRack5 = genSupervisorsWithRacksAndNuma(
+                numRacks, numSupersPerRack, numZonesPerHost, numPortsPerSuper, rackStartNum++, supStartNum,
+                0.0, 0.0, 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);
+
+        List<String> supHostnames = new LinkedList<>();
+        for (SupervisorDetails sup : supMap.values()) {
+            supHostnames.add(sup.getHost());
+        }
+        Map<String, List<String>> rackToHosts = testDNSToSwitchMapping.getRackToHosts();
+        cluster.setNetworkTopography(rackToHosts);
+        System.out.println("DEBUG: rackToHosts: " + rackToHosts.toString());
+
+        NodeSorterHostProximity nodeSorter = new NodeSorterHostProximity(cluster, topo1, BaseResourceAwareStrategy.NodeSortType.DEFAULT_RAS);
+        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"));
+        Assert.assertEquals(rackSummaries + "\n# of racks sorted", 6, sortedRacks.size());
+        Iterator<ObjectResourcesItem> it = sortedRacks.iterator();
+        Assert.assertEquals(rackSummaries + "\nrack-000 should be ordered first since it has the most balanced set of resources", "rack-000", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-001 should be ordered second since it has a balanced set of resources but less than rack-000", "rack-001", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-004 should be ordered third since it has a lot of cpu but not a lot of memory", "rack-004", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-003 should be ordered fourth since it has a lot of memory but not cpu", "rack-003", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nrack-002 should be ordered fifth since it has not cpu resources", "rack-002", it.next().id);
+        Assert.assertEquals(rackSummaries + "\nRack-005 should be ordered sixth since it has neither CPU nor memory available", "rack-005", it.next().id);
+    }
+
+    /**
+     * 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.DEFAULT_RAS);
+        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
+        Assert.assertEquals("rack-000 should be ordered first", "rack-000", it.next().id);
+        // Ranked second since rack-1 has a balanced set of resources but less than rack-0
+        Assert.assertEquals("rack-001 should be ordered second", "rack-001", it.next().id);
+        // Ranked third since rack-4 has a lot of cpu but not a lot of memory
+        Assert.assertEquals("rack-004 should be ordered third", "rack-004", it.next().id);
+        // Ranked fourth since rack-3 has alot of memory but not cpu
+        Assert.assertEquals("rack-003 should be ordered fourth", "rack-003", it.next().id);
+        //Ranked last since rack-2 has not cpu resources
+        Assert.assertEquals("rack-00s2 should be ordered fifth", "rack-002", it.next().id);
+    }
+
+    /**
+     * 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));
+                Assert.fail(err);
+            }
+            seenHosts.add(host);
+            prevHost = host;
+        }
+        System.out.printf("DEBUG: Test Success: Hosts are listed together:\n\t%s\n", String.join("\n\t", errLines));
+    }
+
+    /**
+     * 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());
+        //Assert.assertEquals(rackSummaries + "\n# of racks sorted", 6, sortedRacks.size());
+        Iterator<ObjectResourcesItem> it = sortedRacks.iterator();
+        Assert.assertEquals(topoRequest + "\n\t" + rackSummaries + "\nRack-000 should be ordered first since it has the largest capacity", "rack-000", it.next().id);
+        Assert.assertEquals(topoRequest + "\n\t" + rackSummaries + "\nrack-001 should be ordered second since it smaller than rack-000", "rack-001", it.next().id);
+        Assert.assertEquals(topoRequest + "\n\t" + rackSummaries + "\nrack-002 should be ordered third since it is smaller than rack-001", "rack-002", it.next().id);
+        Assert.assertEquals(topoRequest + "\n\t" + rackSummaries + "\nrack-003 should be ordered fourth since it since it is smaller than rack-002", "rack-003", it.next().id);
+        Assert.assertEquals(topoRequest + "\n\t" + rackSummaries + "\nrack-004 should be ordered fifth since it since it is smaller than rack-003", "rack-004", it.next().id);
+        Assert.assertEquals(topoRequest + "\n\t" + rackSummaries + "\nrack-005 should be ordered last since it since it is has smallest capacity", "rack-005", it.next().id);
+    }
+
+    /**
+     * Schedule two topologies, once with special resources and another without.
+     * There are enough special resources to hold one topology with special resource ("my.gpu").
+     * If the sort order is incorrect, scheduling will not succeed.
+     */
+    @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));
+        //config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, GenericResourceAwareStrategy.class.getName());
+
+        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());
+                Assert.assertEquals(rackSummaries + "\n# of racks sorted", 2, sortedRacks.size());
+                Assert.assertEquals(rackSummaries + "\nFirst rack sorted", "rack-000", sortedRacks.get(0).id);
+                Assert.assertEquals(rackSummaries + "\nSecond rack sorted", "rack-001", sortedRacks.get(1).id);
+            }
+        }
+
+        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);
+        {
+            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());
+                Assert.assertEquals(rackSummaries + "\n# of racks sorted", 2, sortedRacks.size());
+                if (comp.equals("gpu-bolt")) {
+                    Assert.assertEquals(rackSummaries + "\nFirst rack sorted for " + comp, "rack-001", sortedRacks.get(0).id);
+                    Assert.assertEquals(rackSummaries + "\nSecond rack sorted for " + comp, "rack-000", sortedRacks.get(1).id);
+                } else {
+                    Assert.assertEquals(rackSummaries + "\nFirst rack sorted for " + comp, "rack-000", sortedRacks.get(0).id);
+                    Assert.assertEquals(rackSummaries + "\nSecond rack sorted for " + comp, "rack-001", sortedRacks.get(1).id);
+                }
+            }
+        }
+
+        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 (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(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.
+    }
+
+    /**
+     * 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);
+        }
+    }
+
+    private void activateTrace() {
+        List<Class> classesToDebug = Arrays.asList(this.getClass(),
+                BaseResourceAwareStrategy.class, ResourceAwareScheduler.class,
+                NodeSorterHostProximity.class, Cluster.class
+        );
+        Level logLevel = Level.TRACE ; // switch to Level.DEBUG for verbose otherwise Level.INFO
+        classesToDebug.forEach(x -> Configurator.setLevel(x.getName(), logLevel));
+    }
+
+    /**
+     * If the topology is too large for one rack, it should be partially scheduled onto the next rack (and next rack only).
+     */
+    @Test
+    public void testFillUpRackAndSpilloverToNextRack() {
+        //activateTrace();

Review comment:
       Remove if not used?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org