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/02/17 22:08:53 UTC

[GitHub] [storm] bipinprasad opened a new pull request #3379: [STORM-3739] Scheduling should sort numa zones by host groups

bipinprasad opened a new pull request #3379:
URL: https://github.com/apache/storm/pull/3379


   ## What is the purpose of the change
   
   *When using numa zones, nodes should be sorted by host groups for scheduling*
   
   ## How was the change tested
   
   *Added new tests for confirm successful host grouping in storm-server*


----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r583936817



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorter.java
##########
@@ -93,21 +96,15 @@ public NodeSorter(Cluster cluster, TopologyDetails topologyDetails, BaseResource
 
         // from Cluster
         networkTopography = cluster.getNetworkTopography();
-        Map<String, String> hostToRack = new HashMap<>();
-        for (Map.Entry<String, List<String>> entry : networkTopography.entrySet()) {
-            String rackId = entry.getKey();
-            for (String hostName: entry.getValue()) {
-                hostToRack.put(hostName, rackId);
-            }
-        }
+        Map<String, String> hostToRack = cluster.getHostToRack();
         RasNodes nodes = new RasNodes(cluster);
         for (RasNode node: nodes.getNodes()) {
             String superId = node.getId();
             String hostName = node.getHostname();

Review comment:
       Do you want to include the follow up fix on `node.isAlive` in this PR?

##########
File path: storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestDefaultResourceAwareStrategy.java
##########
@@ -905,9 +911,9 @@ public void testMultipleRacksWithFavoritism() {
 
         rs.prepareForScheduling(cluster, topo1);
         INodeSorter nodeSorter = new NodeSorter(cluster, topo1, BaseResourceAwareStrategy.NodeSortType.DEFAULT_RAS);
-        TreeSet<ObjectResourcesItem> sortedRacks= nodeSorter.sortRacks(null);
+        nodeSorter.prepare(null);
+        Iterable<ObjectResourcesItem> sortedRacks= nodeSorter.getSortedRacks();
 
-        Assert.assertEquals("# of racks sorted", 5, sortedRacks.size());

Review comment:
       Why do we want to remove this check?

##########
File path: storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestDefaultResourceAwareStrategy.java
##########
@@ -766,22 +777,17 @@ public void testMultipleRacks() {
         for (Map.Entry<String, String> entry : resolvedSuperVisors.entrySet()) {
             String hostName = entry.getKey();
             String rack = entry.getValue();
-            List<String> nodesForRack = rackToNodes.get(rack);
-            if (nodesForRack == null) {
-                nodesForRack = new ArrayList<>();
-                rackToNodes.put(rack, nodesForRack);
-            }
-            nodesForRack.add(hostName);
+            rackToNodes.computeIfAbsent(rack, rid -> new ArrayList<>()).add(hostName);
         }
         cluster.setNetworkTopography(rackToNodes);
 
         DefaultResourceAwareStrategyOld rs = new DefaultResourceAwareStrategyOld();
         
         rs.prepareForScheduling(cluster, topo1);
-        INodeSorter nodeSorter = new NodeSorter(cluster, topo1, BaseResourceAwareStrategy.NodeSortType.DEFAULT_RAS);
-        TreeSet<ObjectResourcesItem> sortedRacks = nodeSorter.sortRacks(null);
+        INodeSorter nodeSorter = new NodeSorterHostProximity(cluster, topo1, BaseResourceAwareStrategy.NodeSortType.DEFAULT_RAS);
+        nodeSorter.prepare(null);
+        Iterable<ObjectResourcesItem> sortedRacks = nodeSorter.getSortedRacks();
 
-        Assert.assertEquals("# of racks sorted", 6, sortedRacks.size());

Review comment:
       Why do we want to remove this check?




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r592020355



##########
File path: storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestDefaultResourceAwareStrategy.java
##########
@@ -905,9 +911,9 @@ public void testMultipleRacksWithFavoritism() {
 
         rs.prepareForScheduling(cluster, topo1);
         INodeSorter nodeSorter = new NodeSorter(cluster, topo1, BaseResourceAwareStrategy.NodeSortType.DEFAULT_RAS);

Review comment:
       Yes. Changed to use NodeSorterHostProximity.
   
   Rest of the code cleanup of unused classes (after some discussion on NodeSortType) will be in https://issues.apache.org/jira/browse/STORM-3747 




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r593428031



##########
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:
       removed




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582345922



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/ISchedulingState.java
##########
@@ -339,4 +354,31 @@ boolean wouldFit(
      * Get the nimbus configuration.
      */
     Map<String, Object> getConf();
+
+    /**
+     * Determine the list of racks on which topologyIds have been assigned. Note that the returned set
+     * may contain {@link DNSToSwitchMapping#DEFAULT_RACK} if {@link #getHostToRack()} is null or
+     * does not contain the assigned host.
+     *
+     * @param topologyIds for which assignments are examined.
+     * @return set of racks on which assignments have been made.
+     */
+    default Set<String> getAssignedRacks(String... topologyIds) {

Review comment:
       This method gets used in testing for the scheduling. And the right place (based on the interface) seems to be here. Why will developers have to worry, they don't have to implement this method.
   https://docs.oracle.com/javase/tutorial/java/IandI/defaultmethods.html 




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582924807



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResources.java
##########
@@ -384,14 +391,29 @@ public double calculateMinPercentageUsedBy(NormalizedResources used, double tota
 
     /**
      * If a node or rack has a kind of resource not in a request, make that resource negative so when sorting that node or rack will
-     * be less likely to be selected.
+     * be less likely to be selected. If the resource is in the request, make that resource positive.
      * @param request the requested resources.
      */
     public void updateForRareResourceAffinity(NormalizedResources request) {
         int length = Math.min(this.otherResources.length, request.otherResources.length);
         for (int i = 0; i < length; i++) {
-            if (request.getResourceAt(i) == 0.0) {
-                this.otherResources[i] = -1 * this.otherResources[i];
+            if (request.getResourceAt(i) == 0.0 && this.otherResources[i] > 0.0) {
+                this.otherResources[i] = -this.otherResources[i]; // make negative
+            } else if (request.getResourceAt(i) > 0.0 && this.otherResources[i] < 0.0) {

Review comment:
       Thanks




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r581617481



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResources.java
##########
@@ -384,14 +391,29 @@ public double calculateMinPercentageUsedBy(NormalizedResources used, double tota
 
     /**
      * If a node or rack has a kind of resource not in a request, make that resource negative so when sorting that node or rack will
-     * be less likely to be selected.
+     * be less likely to be selected. If the resource is in the request, make that resource positive.
      * @param request the requested resources.
      */
     public void updateForRareResourceAffinity(NormalizedResources request) {
         int length = Math.min(this.otherResources.length, request.otherResources.length);
         for (int i = 0; i < length; i++) {
-            if (request.getResourceAt(i) == 0.0) {
-                this.otherResources[i] = -1 * this.otherResources[i];
+            if (request.getResourceAt(i) == 0.0 && this.otherResources[i] > 0.0) {
+                this.otherResources[i] = -this.otherResources[i]; // make negative
+            } else if (request.getResourceAt(i) > 0.0 && this.otherResources[i] < 0.0) {

Review comment:
       The goal in this code is to make unrequested resource negative (but also requested resource positive). So when processing a new executor with different resource requirements, it can be reapplied and sort resulting elements resorted. With earlier code, this was a "use-once-only" method on the "otherResource". With the change above, the object can be reused.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r593424033



##########
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:
       Removed




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582104768



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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;
+    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 (hostName == null) {
+                // ignore supervisors on blacklistedHosts
+                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 = computeRackToHosts(cluster, superIdToRack);
+
+        // 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;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        Set<String> retVal = new HashSet<>();
+        requestedResources.toNormalizedMap().entrySet().forEach(
+            e -> {
+                if (e.getValue() > 0.0) {
+                    retVal.add(e.getKey());
+                }
+            });
+        return retVal;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.
+     *
+     * @param resourcesSummary     contains all individual {@link ObjectResourcesItem} as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    protected Iterable<ObjectResourcesItem> sortObjectResources(
+            ObjectResourcesSummary resourcesSummary, ExecutorDetails exec, ExistingScheduleFunc existingScheduleFunc) {
+        switch (nodeSortType) {
+            case DEFAULT_RAS:
+                return sortObjectResourcesDefault(resourcesSummary, existingScheduleFunc);
+            case GENERIC_RAS:
+                return sortObjectResourcesGeneric(resourcesSummary, exec, existingScheduleFunc);
+            case COMMON:
+                return sortObjectResourcesCommon(resourcesSummary, exec, existingScheduleFunc);
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Sort objects by the following three criteria.
+     *
+     * <li>
+     *     The number executors of the topology that needs to be scheduled is already on the object (node or rack)
+     *     in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on
+     *     the same object (node or rack) as the existing executors of the topology.
+     * </li>
+     *
+     * <li>
+     *     The subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     *     resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     *     resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     *     How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     *     requested by the executor that the sorting is being done for and pulls it down if it is not.
+     *     By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     *     above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     *     resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     *     have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.
+     *     This is similar to logic used {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     * </li>
+     *
+     * <li>
+     *     The tie between two nodes with same resource availability is broken by using the node with lower minimum
+     *     percentage used. This comparison was used in {@link #sortObjectResourcesDefault(ObjectResourcesSummary, ExistingScheduleFunc)}
+     *     but here it is made subservient to modified resource availbility used in
+     *     {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     *
+     * </li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    private Iterable<ObjectResourcesItem> sortObjectResourcesCommon(
+            final ObjectResourcesSummary allResources, final ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        // Copy and modify allResources
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        final NormalizedResourceRequest requestedResources = (exec != null) ? topologyDetails.getTotalResources(exec) : null;
+        affinityBasedAllResources.getObjectResources().forEach(
+            x -> {
+                if (requestedResources != null) {
+                    // negate unrequested resources
+                    x.availableResources.updateForRareResourceAffinity(requestedResources);
+                }
+                x.minResourcePercent = availableResourcesOverall.calculateMinPercentageUsedBy(x.availableResources);
+                x.avgResourcePercent = availableResourcesOverall.calculateAveragePercentageUsedBy(x.availableResources);
+
+                LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                        x.id, x.minResourcePercent, x.avgResourcePercent,
+                        existingScheduleFunc.getNumExistingSchedule(x.id));
+            }
+        );
+
+        // Use the following comparator to sort
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double o1Avg = o1.avgResourcePercent;
+            double o2Avg = o2.avgResourcePercent;
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     * requested by the executor that the sorting is being done for and pulls it down if it is not.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     * resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     * have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesGeneric(
+            final ObjectResourcesSummary allResources, ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        affinityBasedAllResources.getObjectResources()
+                .forEach(x -> x.availableResources.updateForRareResourceAffinity(requestedResources));
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            double o1Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o1.availableResources);
+            double o2Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o2.availableResources);
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability. So we will be less likely to pick
+     * a rack that have a lot of one resource but a low amount of another.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesDefault(
+            final ObjectResourcesSummary allResources,
+            final ExistingScheduleFunc existingScheduleFunc) {
+
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        for (ObjectResourcesItem objectResources : allResources.getObjectResources()) {
+            objectResources.minResourcePercent =
+                    availableResourcesOverall.calculateMinPercentageUsedBy(objectResources.availableResources);
+            objectResources.avgResourcePercent =
+                    availableResourcesOverall.calculateAveragePercentageUsedBy(objectResources.availableResources);
+            LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                    objectResources.id, objectResources.minResourcePercent, objectResources.avgResourcePercent,
+                    existingScheduleFunc.getNumExistingSchedule(objectResources.id));
+        }
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double diff = o1.avgResourcePercent - o2.avgResourcePercent;
+            if (diff > 0.0) {
+                return -1;
+            } else if (diff < 0.0) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(allResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availHosts a collection of all the hosts we want to sort
+     * @param rackId     the rack id availNodes are a part of
+     * @return an iterable of sorted hosts.
+     */
+    private Iterable<ObjectResourcesItem> sortHosts(
+            Collection<String> availHosts, ExecutorDetails exec, String rackId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary rackResourcesSummary = new ObjectResourcesSummary("RACK");
+        availHosts.forEach(h -> {
+            ObjectResourcesItem hostItem = new ObjectResourcesItem(h);
+            for (RasNode x : hostnameToNodes.get(h)) {
+                hostItem.add(new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0));
+            }
+            rackResourcesSummary.addObjectResourcesItem(hostItem);
+        });
+
+        LOG.debug(
+                "Rack {}: Overall Avail [ {} ] Total [ {} ]",
+                rackId,
+                rackResourcesSummary.getAvailableResourcesOverall(),
+                rackResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            rackResourcesSummary,
+            exec,
+            (hostId) -> {
+                AtomicInteger count = scheduledCount.get(hostId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availRasNodes a list of all the nodes we want to sort
+     * @param hostId     the host-id that availNodes are a part of
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem} for nodes.
+     */
+    private Iterable<ObjectResourcesItem> sortNodes(
+            List<RasNode> availRasNodes, ExecutorDetails exec, String hostId,
+            Map<String, AtomicInteger> scheduledCount) {

Review comment:
       It feels like an overuse of AtomicInteger.  If this is only about reducing the code, I would suggest to use getOrDefault method, for example,
   
   ```
   perNodeScheduledCount.computeIfAbsent(superId, (sid) -> new AtomicInteger(0))
                           .getAndAdd(entry.getValue().size());
   ```
   can be changed to 
   
   ```
   perNodeScheduledCount.put(superId, perNodeScheduledCount.getOrDefault(superId, 0) + entry.getValue().size());
   ```
   And 
   ```
   (rackId) -> {
                   AtomicInteger count = scheduledCount.get(rackId);
                   if (count == null) {
                       return 0;
                   }
                   return count.get();
               }
   ```
   can be changed to 
   ```
   (rackId) -> scheduledCount.getOrDefault(rackId, 0)
   ```




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r593425686



##########
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:
       fixed
   




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r593424282



##########
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:
       Removed




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r581620458



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/BaseResourceAwareStrategy.java
##########
@@ -189,7 +207,7 @@ protected void prepareForScheduling(Cluster cluster, TopologyDetails topologyDet
         LOG.debug("The max state search that will be used by topology {} is {}", topologyDetails.getId(), maxStateSearch);
 
         searcherState = createSearcherState();
-        setNodeSorter(new NodeSorter(cluster, topologyDetails, nodeSortType));
+        setNodeSorter(new NodeSorterHostProximity(cluster, topologyDetails));

Review comment:
       Yes, only sorts one way.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582347447



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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 {

Review comment:
       I will create a separate Jira.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582091558



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/ISchedulingState.java
##########
@@ -339,4 +354,31 @@ boolean wouldFit(
      * Get the nimbus configuration.
      */
     Map<String, Object> getConf();
+
+    /**
+     * Determine the list of racks on which topologyIds have been assigned. Note that the returned set
+     * may contain {@link DNSToSwitchMapping#DEFAULT_RACK} if {@link #getHostToRack()} is null or
+     * does not contain the assigned host.
+     *
+     * @param topologyIds for which assignments are examined.
+     * @return set of racks on which assignments have been made.
+     */
+    default Set<String> getAssignedRacks(String... topologyIds) {

Review comment:
       Does it make sense to delete this from this PR and move it to the PR for defragmentation code? It makes it easier to understand when looking back the commit history in the future




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r581619307



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/ISchedulingState.java
##########
@@ -339,4 +354,31 @@ boolean wouldFit(
      * Get the nimbus configuration.
      */
     Map<String, Object> getConf();
+
+    /**
+     * Determine the list of racks on which topologyIds have been assigned. Note that the returned set
+     * may contain {@link DNSToSwitchMapping#DEFAULT_RACK} if {@link #getHostToRack()} is null or
+     * does not contain the assigned host.
+     *
+     * @param topologyIds for which assignments are examined.
+     * @return set of racks on which assignments have been made.
+     */
+    default Set<String> getAssignedRacks(String... topologyIds) {

Review comment:
       This will be used in defragmentation code.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r593427347



##########
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:
       removed.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r583954251



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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;
+    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 (hostName == null) {
+                // ignore supervisors on blacklistedHosts
+                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 = computeRackToHosts(cluster, superIdToRack);
+
+        // 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;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        Set<String> retVal = new HashSet<>();
+        requestedResources.toNormalizedMap().entrySet().forEach(
+            e -> {
+                if (e.getValue() > 0.0) {
+                    retVal.add(e.getKey());
+                }
+            });
+        return retVal;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.
+     *
+     * @param resourcesSummary     contains all individual {@link ObjectResourcesItem} as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    protected Iterable<ObjectResourcesItem> sortObjectResources(
+            ObjectResourcesSummary resourcesSummary, ExecutorDetails exec, ExistingScheduleFunc existingScheduleFunc) {
+        switch (nodeSortType) {
+            case DEFAULT_RAS:
+                return sortObjectResourcesDefault(resourcesSummary, existingScheduleFunc);
+            case GENERIC_RAS:
+                return sortObjectResourcesGeneric(resourcesSummary, exec, existingScheduleFunc);
+            case COMMON:
+                return sortObjectResourcesCommon(resourcesSummary, exec, existingScheduleFunc);
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Sort objects by the following three criteria.
+     *
+     * <li>
+     *     The number executors of the topology that needs to be scheduled is already on the object (node or rack)
+     *     in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on
+     *     the same object (node or rack) as the existing executors of the topology.
+     * </li>
+     *
+     * <li>
+     *     The subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     *     resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     *     resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     *     How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     *     requested by the executor that the sorting is being done for and pulls it down if it is not.
+     *     By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     *     above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     *     resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     *     have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.
+     *     This is similar to logic used {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     * </li>
+     *
+     * <li>
+     *     The tie between two nodes with same resource availability is broken by using the node with lower minimum
+     *     percentage used. This comparison was used in {@link #sortObjectResourcesDefault(ObjectResourcesSummary, ExistingScheduleFunc)}
+     *     but here it is made subservient to modified resource availbility used in
+     *     {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     *
+     * </li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    private Iterable<ObjectResourcesItem> sortObjectResourcesCommon(
+            final ObjectResourcesSummary allResources, final ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        // Copy and modify allResources
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        final NormalizedResourceRequest requestedResources = (exec != null) ? topologyDetails.getTotalResources(exec) : null;
+        affinityBasedAllResources.getObjectResources().forEach(
+            x -> {
+                if (requestedResources != null) {
+                    // negate unrequested resources
+                    x.availableResources.updateForRareResourceAffinity(requestedResources);
+                }
+                x.minResourcePercent = availableResourcesOverall.calculateMinPercentageUsedBy(x.availableResources);
+                x.avgResourcePercent = availableResourcesOverall.calculateAveragePercentageUsedBy(x.availableResources);
+
+                LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                        x.id, x.minResourcePercent, x.avgResourcePercent,
+                        existingScheduleFunc.getNumExistingSchedule(x.id));
+            }
+        );
+
+        // Use the following comparator to sort
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double o1Avg = o1.avgResourcePercent;
+            double o2Avg = o2.avgResourcePercent;
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     * requested by the executor that the sorting is being done for and pulls it down if it is not.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     * resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     * have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesGeneric(
+            final ObjectResourcesSummary allResources, ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        affinityBasedAllResources.getObjectResources()
+                .forEach(x -> x.availableResources.updateForRareResourceAffinity(requestedResources));
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            double o1Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o1.availableResources);
+            double o2Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o2.availableResources);
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability. So we will be less likely to pick
+     * a rack that have a lot of one resource but a low amount of another.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesDefault(
+            final ObjectResourcesSummary allResources,
+            final ExistingScheduleFunc existingScheduleFunc) {
+
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        for (ObjectResourcesItem objectResources : allResources.getObjectResources()) {
+            objectResources.minResourcePercent =
+                    availableResourcesOverall.calculateMinPercentageUsedBy(objectResources.availableResources);
+            objectResources.avgResourcePercent =
+                    availableResourcesOverall.calculateAveragePercentageUsedBy(objectResources.availableResources);
+            LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                    objectResources.id, objectResources.minResourcePercent, objectResources.avgResourcePercent,
+                    existingScheduleFunc.getNumExistingSchedule(objectResources.id));
+        }
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double diff = o1.avgResourcePercent - o2.avgResourcePercent;
+            if (diff > 0.0) {
+                return -1;
+            } else if (diff < 0.0) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(allResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availHosts a collection of all the hosts we want to sort
+     * @param rackId     the rack id availNodes are a part of
+     * @return an iterable of sorted hosts.
+     */
+    private Iterable<ObjectResourcesItem> sortHosts(
+            Collection<String> availHosts, ExecutorDetails exec, String rackId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary rackResourcesSummary = new ObjectResourcesSummary("RACK");
+        availHosts.forEach(h -> {
+            ObjectResourcesItem hostItem = new ObjectResourcesItem(h);
+            for (RasNode x : hostnameToNodes.get(h)) {
+                hostItem.add(new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0));
+            }
+            rackResourcesSummary.addObjectResourcesItem(hostItem);
+        });
+
+        LOG.debug(
+                "Rack {}: Overall Avail [ {} ] Total [ {} ]",
+                rackId,
+                rackResourcesSummary.getAvailableResourcesOverall(),
+                rackResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            rackResourcesSummary,
+            exec,
+            (hostId) -> {
+                AtomicInteger count = scheduledCount.get(hostId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availRasNodes a list of all the nodes we want to sort
+     * @param hostId     the host-id that availNodes are a part of
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem} for nodes.
+     */
+    private Iterable<ObjectResourcesItem> sortNodes(
+            List<RasNode> availRasNodes, ExecutorDetails exec, String hostId,
+            Map<String, AtomicInteger> scheduledCount) {

Review comment:
       Although I still have different opinions after offline discussion, I won't block this PR on this.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r580773481



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/ISchedulingState.java
##########
@@ -282,6 +285,18 @@ boolean wouldFit(
      */
     Map<String, List<String>> getNetworkTopography();
 
+    /**
+     * Get host -> rack map - the inverse of networkTopography..

Review comment:
       removed




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r584026011



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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;
+    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 (hostName == null) {
+                // ignore supervisors on blacklistedHosts
+                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 = computeRackToHosts(cluster, superIdToRack);
+
+        // 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;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {

Review comment:
       Now that TestLargeCluster tests multiple clusters, I wish it were faster. But not in this PR.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582326264



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResources.java
##########
@@ -384,14 +391,29 @@ public double calculateMinPercentageUsedBy(NormalizedResources used, double tota
 
     /**
      * If a node or rack has a kind of resource not in a request, make that resource negative so when sorting that node or rack will
-     * be less likely to be selected.
+     * be less likely to be selected. If the resource is in the request, make that resource positive.
      * @param request the requested resources.
      */
     public void updateForRareResourceAffinity(NormalizedResources request) {
         int length = Math.min(this.otherResources.length, request.otherResources.length);
         for (int i = 0; i < length; i++) {
-            if (request.getResourceAt(i) == 0.0) {
-                this.otherResources[i] = -1 * this.otherResources[i];
+            if (request.getResourceAt(i) == 0.0 && this.otherResources[i] > 0.0) {
+                this.otherResources[i] = -this.otherResources[i]; // make negative
+            } else if (request.getResourceAt(i) > 0.0 && this.otherResources[i] < 0.0) {

Review comment:
       I am not sure this has been reported or not.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r580371328



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResources.java
##########
@@ -384,14 +391,29 @@ public double calculateMinPercentageUsedBy(NormalizedResources used, double tota
 
     /**
      * If a node or rack has a kind of resource not in a request, make that resource negative so when sorting that node or rack will
-     * be less likely to be selected.
+     * be less likely to be selected. If the resource is in the request, make that resource positive.
      * @param request the requested resources.
      */
     public void updateForRareResourceAffinity(NormalizedResources request) {
         int length = Math.min(this.otherResources.length, request.otherResources.length);
         for (int i = 0; i < length; i++) {
-            if (request.getResourceAt(i) == 0.0) {
-                this.otherResources[i] = -1 * this.otherResources[i];
+            if (request.getResourceAt(i) == 0.0 && this.otherResources[i] > 0.0) {
+                this.otherResources[i] = -this.otherResources[i]; // make negative
+            } else if (request.getResourceAt(i) > 0.0 && this.otherResources[i] < 0.0) {
+                this.otherResources[i] = -this.otherResources[i]; // make positive
+            }
+        }
+    }
+
+    /**
+     * If the resource is negative, then make that resource positive.
+     * This will undo effects of {@link #updateForRareResourceAffinity(NormalizedResources)}.
+     */
+    public void revertUpdatesForRareResourceAffinity() {

Review comment:
       Can we remove this? It doesn't seem to be used anywhere 

##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResources.java
##########
@@ -384,14 +391,29 @@ public double calculateMinPercentageUsedBy(NormalizedResources used, double tota
 
     /**
      * If a node or rack has a kind of resource not in a request, make that resource negative so when sorting that node or rack will
-     * be less likely to be selected.
+     * be less likely to be selected. If the resource is in the request, make that resource positive.
      * @param request the requested resources.
      */
     public void updateForRareResourceAffinity(NormalizedResources request) {
         int length = Math.min(this.otherResources.length, request.otherResources.length);
         for (int i = 0; i < length; i++) {
-            if (request.getResourceAt(i) == 0.0) {
-                this.otherResources[i] = -1 * this.otherResources[i];
+            if (request.getResourceAt(i) == 0.0 && this.otherResources[i] > 0.0) {
+                this.otherResources[i] = -this.otherResources[i]; // make negative
+            } else if (request.getResourceAt(i) > 0.0 && this.otherResources[i] < 0.0) {

Review comment:
       Can you please help me understand why is this change necessary? Did you find some bugs here? Thanks

##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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;
+    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 (hostName == null) {
+                // ignore supervisors on blacklistedHosts
+                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 = computeRackToHosts(cluster, superIdToRack);
+
+        // 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;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {

Review comment:
       Seems not used anywhere. Can we delete it?

##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,717 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.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, List<RasNode>> rackIdToNodes = new HashMap<>();
+    private final Map<String, List<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();
+        Map<String, String> hostToRack = cluster.getHostToRack();
+        RasNodes nodes = new RasNodes(cluster);
+        for (RasNode node: nodes.getNodes()) {
+            String superId = node.getId();
+            String hostName = node.getHostname();
+            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, id -> new ArrayList<>()).add(hostName);
+        }
+        this.greyListedSupervisorIds = cluster.getGreyListedSupervisors();
+
+        // 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);
+    }
+
+    @Override
+    public void prepare(ExecutorDetails exec) {
+        this.exec = exec;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        Set<String> retVal = new HashSet<>();
+        requestedResources.toNormalizedMap().entrySet().forEach(
+            e -> {
+                if (e.getValue() > 0.0) {
+                    retVal.add(e.getKey());
+                }
+            });
+        return retVal;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.
+     *
+     * @param resourcesSummary     contains all individual {@link ObjectResourcesItem} as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    protected Iterable<ObjectResourcesItem> sortObjectResources(
+            ObjectResourcesSummary resourcesSummary, ExecutorDetails exec, ExistingScheduleFunc existingScheduleFunc) {
+        switch (nodeSortType) {
+            case DEFAULT_RAS:
+                return sortObjectResourcesDefault(resourcesSummary, existingScheduleFunc);
+            case GENERIC_RAS:
+                return sortObjectResourcesGeneric(resourcesSummary, exec, existingScheduleFunc);
+            case COMMON:
+                return sortObjectResourcesCommon(resourcesSummary, exec, existingScheduleFunc);
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Sort objects by the following three criteria.
+     *
+     * <li>
+     *     The number executors of the topology that needs to be scheduled is already on the object (node or rack)
+     *     in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on
+     *     the same object (node or rack) as the existing executors of the topology.
+     * </li>
+     *
+     * <li>
+     *     The subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     *     resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     *     resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     *     How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     *     requested by the executor that the sorting is being done for and pulls it down if it is not.
+     *     By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     *     above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     *     resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     *     have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.
+     *     This is similar to logic used {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     * </li>
+     *
+     * <li>
+     *     The tie between two nodes with same resource availability is broken by using the node with lower minimum
+     *     percentage used. This comparison was used in {@link #sortObjectResourcesDefault(ObjectResourcesSummary, ExistingScheduleFunc)}
+     *     but here it is made subservient to modified resource availbility used in
+     *     {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     *
+     * </li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    private Iterable<ObjectResourcesItem> sortObjectResourcesCommon(
+            final ObjectResourcesSummary allResources, final ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        // Copy and modify allResources
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        final NormalizedResourceRequest requestedResources = (exec != null) ? topologyDetails.getTotalResources(exec) : null;
+        affinityBasedAllResources.getObjectResources().forEach(
+            x -> {
+                if (requestedResources != null) {
+                    // negate unrequested resources
+                    x.availableResources.updateForRareResourceAffinity(requestedResources);
+                }
+                x.minResourcePercent = availableResourcesOverall.calculateMinPercentageUsedBy(x.availableResources);
+                x.avgResourcePercent = availableResourcesOverall.calculateAveragePercentageUsedBy(x.availableResources);
+
+                LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                        x.id, x.minResourcePercent, x.avgResourcePercent,
+                        existingScheduleFunc.getNumExistingSchedule(x.id));
+            }
+        );
+
+        // Use the following comparator to sort
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double o1Avg = o1.avgResourcePercent;
+            double o2Avg = o2.avgResourcePercent;
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     * requested by the executor that the sorting is being done for and pulls it down if it is not.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     * resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     * have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesGeneric(
+            final ObjectResourcesSummary allResources, ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        affinityBasedAllResources.getObjectResources()
+                .forEach(x -> x.availableResources.updateForRareResourceAffinity(requestedResources));
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            double o1Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o1.availableResources);
+            double o2Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o2.availableResources);
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability. So we will be less likely to pick
+     * a rack that have a lot of one resource but a low amount of another.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesDefault(
+            final ObjectResourcesSummary allResources,
+            final ExistingScheduleFunc existingScheduleFunc) {
+
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        for (ObjectResourcesItem objectResources : allResources.getObjectResources()) {
+            objectResources.minResourcePercent =
+                    availableResourcesOverall.calculateMinPercentageUsedBy(objectResources.availableResources);
+            objectResources.avgResourcePercent =
+                    availableResourcesOverall.calculateAveragePercentageUsedBy(objectResources.availableResources);
+            LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                    objectResources.id, objectResources.minResourcePercent, objectResources.avgResourcePercent,
+                    existingScheduleFunc.getNumExistingSchedule(objectResources.id));
+        }
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double diff = o1.avgResourcePercent - o2.avgResourcePercent;
+            if (diff > 0.0) {
+                return -1;
+            } else if (diff < 0.0) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(allResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availHosts a list of all the hosts we want to sort
+     * @param rackId     the rack id availNodes are a part of
+     * @return an iterable of sorted hosts.
+     */
+    private Iterable<ObjectResourcesItem> sortHosts(
+            List<String> availHosts, ExecutorDetails exec, String rackId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary rackResourcesSummary = new ObjectResourcesSummary("RACK");
+        availHosts.forEach(h -> {
+            ObjectResourcesItem hostItem = new ObjectResourcesItem(h);
+            for (RasNode x : hostnameToNodes.get(h)) {
+                hostItem.add(new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0));
+            }
+            rackResourcesSummary.addObjectResourcesItem(hostItem);
+        });
+
+        LOG.debug(
+                "Rack {}: Overall Avail [ {} ] Total [ {} ]",
+                rackId,
+                rackResourcesSummary.getAvailableResourcesOverall(),
+                rackResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            rackResourcesSummary,
+            exec,
+            (hostId) -> {
+                AtomicInteger count = scheduledCount.get(hostId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availRasNodes a list of all the nodes we want to sort
+     * @param hostId     the host-id that availNodes are a part of
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem} for nodes.
+     */
+    private Iterable<ObjectResourcesItem> sortNodes(
+            List<RasNode> availRasNodes, ExecutorDetails exec, String hostId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary hostResourcesSummary = new ObjectResourcesSummary("HOST");
+        availRasNodes.forEach(x ->
+                hostResourcesSummary.addObjectResourcesItem(
+                        new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0)
+                )
+        );
+
+        LOG.debug(
+            "Host {}: Overall Avail [ {} ] Total [ {} ]",
+            hostId,
+            hostResourcesSummary.getAvailableResourcesOverall(),
+            hostResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            hostResourcesSummary,
+            exec,
+            (superId) -> {
+                AtomicInteger count = scheduledCount.get(superId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    protected List<String> makeHostToNodeIds(List<String> hosts) {
+        if (hosts == null) {
+            return Collections.emptyList();
+        }
+        List<String> ret = new ArrayList<>(hosts.size());
+        for (String host: hosts) {
+            List<RasNode> nodes = hostnameToNodes.get(host);
+            if (nodes != null) {
+                for (RasNode node : nodes) {
+                    ret.add(node.getId());
+                }
+            }
+        }
+        return ret;
+    }
+
+    private class LazyNodeSortingIterator implements Iterator<String> {
+        private final LazyNodeSorting parent;
+        private final Iterator<ObjectResourcesItem> rackIterator;
+        private Iterator<ObjectResourcesItem> hostIterator;
+        private Iterator<ObjectResourcesItem> nodeIterator;
+        private String nextValueFromNode = null;
+        private final Iterator<String> pre;
+        private final Iterator<String> post;
+        private final Set<String> skip;
+
+        LazyNodeSortingIterator(LazyNodeSorting parent, Iterable<ObjectResourcesItem> sortedRacks) {
+            this.parent = parent;
+            rackIterator = sortedRacks.iterator();
+            pre = favoredNodeIds.iterator();
+            post = Stream.concat(unFavoredNodeIds.stream(), greyListedSupervisorIds.stream())
+                            .collect(Collectors.toList())
+                            .iterator();
+            skip = parent.skippedNodeIds;
+        }
+
+        private Iterator<ObjectResourcesItem> getNodeIterator() {
+            if (nodeIterator != null && nodeIterator.hasNext()) {
+                return nodeIterator;
+            }
+            //need to get the next host/node iterator
+            if (hostIterator != null && hostIterator.hasNext()) {
+                ObjectResourcesItem host = hostIterator.next();
+                final String hostId = host.id;
+                nodeIterator = parent.getSortedNodesForHost(hostId).iterator();
+                return nodeIterator;
+            }
+            if (rackIterator.hasNext()) {
+                ObjectResourcesItem rack = rackIterator.next();
+                final String rackId = rack.id;
+                hostIterator = parent.getSortedHostsForRack(rackId).iterator();
+                ObjectResourcesItem host = hostIterator.next();
+                final String hostId = host.id;
+                nodeIterator = parent.getSortedNodesForHost(hostId).iterator();
+                return nodeIterator;
+            }
+
+            return null;
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (pre.hasNext()) {
+                return true;
+            }
+            if (nextValueFromNode != null) {
+                return true;
+            }
+            while (true) {
+                //For the node we don't know if we have another one unless we look at the contents
+                Iterator<ObjectResourcesItem> nodeIterator = getNodeIterator();
+                if (nodeIterator == null || !nodeIterator.hasNext()) {
+                    break;
+                }
+                String tmp = nodeIterator.next().id;
+                if (!skip.contains(tmp)) {
+                    nextValueFromNode = tmp;
+                    return true;
+                }
+            }
+            return post.hasNext();
+        }
+
+        @Override
+        public String next() {
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            if (pre.hasNext()) {
+                return pre.next();
+            }
+            if (nextValueFromNode != null) {
+                String tmp = nextValueFromNode;
+                nextValueFromNode = null;
+                return tmp;
+            }
+            return post.next();
+        }
+    }
+
+    private class LazyNodeSorting implements Iterable<String> {
+        private final Map<String, AtomicInteger> perHostScheduledCount = new HashMap<>();
+        private final Map<String, AtomicInteger> perNodeScheduledCount = new HashMap<>();
+        private final Iterable<ObjectResourcesItem> sortedRacks;
+        private final Map<String, Iterable<ObjectResourcesItem>> cachedHosts = new HashMap<>();
+        private final Map<String, Iterable<ObjectResourcesItem>> cachedNodesByHost = new HashMap<>();
+        private final ExecutorDetails exec;
+        private final Set<String> skippedNodeIds = new HashSet<>();
+
+        LazyNodeSorting(ExecutorDetails exec) {
+            this.exec = exec;
+            skippedNodeIds.addAll(favoredNodeIds);
+            skippedNodeIds.addAll(unFavoredNodeIds);
+            skippedNodeIds.addAll(greyListedSupervisorIds);
+
+            String topoId = topologyDetails.getId();
+            SchedulerAssignment assignment = cluster.getAssignmentById(topoId);
+            if (assignment != null) {
+                for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> entry :
+                    assignment.getSlotToExecutors().entrySet()) {
+                    String superId = entry.getKey().getNodeId();
+                    String hostId = nodeIdToHostname.get(superId);
+                    perHostScheduledCount.computeIfAbsent(hostId, id -> new AtomicInteger(0))
+                        .getAndAdd(entry.getValue().size());
+                    perNodeScheduledCount.computeIfAbsent(superId, id -> new AtomicInteger(0))
+                        .getAndAdd(entry.getValue().size());
+                }
+            }
+            sortedRacks = getSortedRacks();
+        }
+
+        private Iterable<ObjectResourcesItem> getSortedHostsForRack(String rackId) {
+            return cachedHosts.computeIfAbsent(rackId,
+                id -> sortHosts(rackIdToHosts.getOrDefault(id, Collections.emptyList()), exec, id, perHostScheduledCount));
+        }
+
+        private Iterable<ObjectResourcesItem> getSortedNodesForHost(String hostId) {
+            return cachedNodesByHost.computeIfAbsent(hostId,
+                id -> sortNodes(hostnameToNodes.getOrDefault(id, Collections.emptyList()), exec, id, perNodeScheduledCount));
+        }
+
+        @Override
+        public Iterator<String> iterator() {
+            return new LazyNodeSortingIterator(this, sortedRacks);
+        }
+    }
+
+    @Override
+    public Iterable<String> sortAllNodes() {
+        //LOG.info("sortAllNodes():cachedLazyNodeIterators.size={}, execRequestResourceKey={}",
+        //            cachedLazyNodeIterators.size(), execRequestResourcesKey);
+        //return cachedLazyNodeIterators.computeIfAbsent(execRequestResourcesKey, k -> new LazyNodeSorting(exec));
+        return new LazyNodeSorting(exec);
+    }
+
+    private ObjectResourcesSummary createClusterSummarizedResources() {
+        ObjectResourcesSummary clusterResourcesSummary = new ObjectResourcesSummary("Cluster");
+
+        //This is the first time so initialize the resources.
+        for (Map.Entry<String, List<String>> entry : networkTopography.entrySet()) {
+            String rackId = entry.getKey();
+            List<String> nodeHosts = entry.getValue();
+            ObjectResourcesItem rack = new ObjectResourcesItem(rackId);
+            for (String nodeHost : nodeHosts) {
+                for (RasNode node : hostnameToNodes(nodeHost)) {
+                    rack.availableResources.add(node.getTotalAvailableResources());
+                    rack.totalResources.add(node.getTotalAvailableResources());

Review comment:
       Should this be `node.getTotalResources()`?

##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/ISchedulingState.java
##########
@@ -339,4 +354,31 @@ boolean wouldFit(
      * Get the nimbus configuration.
      */
     Map<String, Object> getConf();
+
+    /**
+     * Determine the list of racks on which topologyIds have been assigned. Note that the returned set
+     * may contain {@link DNSToSwitchMapping#DEFAULT_RACK} if {@link #getHostToRack()} is null or
+     * does not contain the assigned host.
+     *
+     * @param topologyIds for which assignments are examined.
+     * @return set of racks on which assignments have been made.
+     */
+    default Set<String> getAssignedRacks(String... topologyIds) {

Review comment:
       This seems only used in test cases. Do we want to move it into some test classes, instead of adding it in the interface? So the interface can be smaller and developers don't need to worry about this method

##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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;
+    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 (hostName == null) {
+                // ignore supervisors on blacklistedHosts
+                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 = computeRackToHosts(cluster, superIdToRack);
+
+        // 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;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        Set<String> retVal = new HashSet<>();
+        requestedResources.toNormalizedMap().entrySet().forEach(
+            e -> {
+                if (e.getValue() > 0.0) {
+                    retVal.add(e.getKey());
+                }
+            });
+        return retVal;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.
+     *
+     * @param resourcesSummary     contains all individual {@link ObjectResourcesItem} as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    protected Iterable<ObjectResourcesItem> sortObjectResources(
+            ObjectResourcesSummary resourcesSummary, ExecutorDetails exec, ExistingScheduleFunc existingScheduleFunc) {
+        switch (nodeSortType) {
+            case DEFAULT_RAS:
+                return sortObjectResourcesDefault(resourcesSummary, existingScheduleFunc);
+            case GENERIC_RAS:
+                return sortObjectResourcesGeneric(resourcesSummary, exec, existingScheduleFunc);
+            case COMMON:
+                return sortObjectResourcesCommon(resourcesSummary, exec, existingScheduleFunc);
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Sort objects by the following three criteria.
+     *
+     * <li>
+     *     The number executors of the topology that needs to be scheduled is already on the object (node or rack)
+     *     in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on
+     *     the same object (node or rack) as the existing executors of the topology.
+     * </li>
+     *
+     * <li>
+     *     The subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     *     resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     *     resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     *     How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     *     requested by the executor that the sorting is being done for and pulls it down if it is not.
+     *     By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     *     above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     *     resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     *     have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.
+     *     This is similar to logic used {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     * </li>
+     *
+     * <li>
+     *     The tie between two nodes with same resource availability is broken by using the node with lower minimum
+     *     percentage used. This comparison was used in {@link #sortObjectResourcesDefault(ObjectResourcesSummary, ExistingScheduleFunc)}
+     *     but here it is made subservient to modified resource availbility used in
+     *     {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     *
+     * </li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    private Iterable<ObjectResourcesItem> sortObjectResourcesCommon(
+            final ObjectResourcesSummary allResources, final ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        // Copy and modify allResources
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        final NormalizedResourceRequest requestedResources = (exec != null) ? topologyDetails.getTotalResources(exec) : null;
+        affinityBasedAllResources.getObjectResources().forEach(
+            x -> {
+                if (requestedResources != null) {
+                    // negate unrequested resources
+                    x.availableResources.updateForRareResourceAffinity(requestedResources);
+                }
+                x.minResourcePercent = availableResourcesOverall.calculateMinPercentageUsedBy(x.availableResources);
+                x.avgResourcePercent = availableResourcesOverall.calculateAveragePercentageUsedBy(x.availableResources);
+
+                LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                        x.id, x.minResourcePercent, x.avgResourcePercent,
+                        existingScheduleFunc.getNumExistingSchedule(x.id));
+            }
+        );
+
+        // Use the following comparator to sort
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double o1Avg = o1.avgResourcePercent;
+            double o2Avg = o2.avgResourcePercent;
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     * requested by the executor that the sorting is being done for and pulls it down if it is not.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     * resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     * have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesGeneric(
+            final ObjectResourcesSummary allResources, ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        affinityBasedAllResources.getObjectResources()
+                .forEach(x -> x.availableResources.updateForRareResourceAffinity(requestedResources));
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            double o1Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o1.availableResources);
+            double o2Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o2.availableResources);
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability. So we will be less likely to pick
+     * a rack that have a lot of one resource but a low amount of another.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesDefault(
+            final ObjectResourcesSummary allResources,
+            final ExistingScheduleFunc existingScheduleFunc) {
+
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        for (ObjectResourcesItem objectResources : allResources.getObjectResources()) {
+            objectResources.minResourcePercent =
+                    availableResourcesOverall.calculateMinPercentageUsedBy(objectResources.availableResources);
+            objectResources.avgResourcePercent =
+                    availableResourcesOverall.calculateAveragePercentageUsedBy(objectResources.availableResources);
+            LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                    objectResources.id, objectResources.minResourcePercent, objectResources.avgResourcePercent,
+                    existingScheduleFunc.getNumExistingSchedule(objectResources.id));
+        }
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double diff = o1.avgResourcePercent - o2.avgResourcePercent;
+            if (diff > 0.0) {
+                return -1;
+            } else if (diff < 0.0) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(allResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availHosts a collection of all the hosts we want to sort
+     * @param rackId     the rack id availNodes are a part of
+     * @return an iterable of sorted hosts.
+     */
+    private Iterable<ObjectResourcesItem> sortHosts(
+            Collection<String> availHosts, ExecutorDetails exec, String rackId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary rackResourcesSummary = new ObjectResourcesSummary("RACK");
+        availHosts.forEach(h -> {
+            ObjectResourcesItem hostItem = new ObjectResourcesItem(h);
+            for (RasNode x : hostnameToNodes.get(h)) {
+                hostItem.add(new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0));
+            }
+            rackResourcesSummary.addObjectResourcesItem(hostItem);
+        });
+
+        LOG.debug(
+                "Rack {}: Overall Avail [ {} ] Total [ {} ]",
+                rackId,
+                rackResourcesSummary.getAvailableResourcesOverall(),
+                rackResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            rackResourcesSummary,
+            exec,
+            (hostId) -> {
+                AtomicInteger count = scheduledCount.get(hostId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availRasNodes a list of all the nodes we want to sort
+     * @param hostId     the host-id that availNodes are a part of
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem} for nodes.
+     */
+    private Iterable<ObjectResourcesItem> sortNodes(
+            List<RasNode> availRasNodes, ExecutorDetails exec, String hostId,
+            Map<String, AtomicInteger> scheduledCount) {

Review comment:
       Can you please help me understand why AtomicInteger is required? My understanding is that scheduling is done sequentially in one thread. 

##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/BaseResourceAwareStrategy.java
##########
@@ -189,7 +207,7 @@ protected void prepareForScheduling(Cluster cluster, TopologyDetails topologyDet
         LOG.debug("The max state search that will be used by topology {} is {}", topologyDetails.getId(), maxStateSearch);
 
         searcherState = createSearcherState();
-        setNodeSorter(new NodeSorter(cluster, topologyDetails, nodeSortType));
+        setNodeSorter(new NodeSorterHostProximity(cluster, topologyDetails));

Review comment:
       Do we not care about `nodeSortType` in this new `NodeSorterHostProximity` class?

##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/ISchedulingState.java
##########
@@ -282,6 +285,18 @@ boolean wouldFit(
      */
     Map<String, List<String>> getNetworkTopography();
 
+    /**
+     * Get host -> rack map - the inverse of networkTopography.
+     */
+    default Map<String, String> getHostToRack() {

Review comment:
       Since this is calculated based on getNetworkTopography,  do we want to cache it in the implementation class like `networkTopography` instead of recomputing it every time?

##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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 {

Review comment:
       This class looks very similar to NodeSorter. Why do we want to keep both of them? And if we need to keep both of them, does it make sense to have NodeSorterHostProximity extend NodeSorter?




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r583810797



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResources.java
##########
@@ -384,14 +391,29 @@ public double calculateMinPercentageUsedBy(NormalizedResources used, double tota
 
     /**
      * If a node or rack has a kind of resource not in a request, make that resource negative so when sorting that node or rack will
-     * be less likely to be selected.
+     * be less likely to be selected. If the resource is in the request, make that resource positive.
      * @param request the requested resources.
      */
     public void updateForRareResourceAffinity(NormalizedResources request) {
         int length = Math.min(this.otherResources.length, request.otherResources.length);
         for (int i = 0; i < length; i++) {
-            if (request.getResourceAt(i) == 0.0) {
-                this.otherResources[i] = -1 * this.otherResources[i];
+            if (request.getResourceAt(i) == 0.0 && this.otherResources[i] > 0.0) {
+                this.otherResources[i] = -this.otherResources[i]; // make negative
+            } else if (request.getResourceAt(i) > 0.0 && this.otherResources[i] < 0.0) {
+                this.otherResources[i] = -this.otherResources[i]; // make positive
+            }
+        }
+    }
+
+    /**
+     * If the resource is negative, then make that resource positive.
+     * This will undo effects of {@link #updateForRareResourceAffinity(NormalizedResources)}.
+     */
+    public void revertUpdatesForRareResourceAffinity() {

Review comment:
       Will remove from this PR.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582850780



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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 {

Review comment:
       https://issues.apache.org/jira/browse/STORM-3747 for removal of deprecated and unused classes




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r581622323



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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;
+    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 (hostName == null) {
+                // ignore supervisors on blacklistedHosts
+                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 = computeRackToHosts(cluster, superIdToRack);
+
+        // 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;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        Set<String> retVal = new HashSet<>();
+        requestedResources.toNormalizedMap().entrySet().forEach(
+            e -> {
+                if (e.getValue() > 0.0) {
+                    retVal.add(e.getKey());
+                }
+            });
+        return retVal;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.
+     *
+     * @param resourcesSummary     contains all individual {@link ObjectResourcesItem} as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    protected Iterable<ObjectResourcesItem> sortObjectResources(
+            ObjectResourcesSummary resourcesSummary, ExecutorDetails exec, ExistingScheduleFunc existingScheduleFunc) {
+        switch (nodeSortType) {
+            case DEFAULT_RAS:
+                return sortObjectResourcesDefault(resourcesSummary, existingScheduleFunc);
+            case GENERIC_RAS:
+                return sortObjectResourcesGeneric(resourcesSummary, exec, existingScheduleFunc);
+            case COMMON:
+                return sortObjectResourcesCommon(resourcesSummary, exec, existingScheduleFunc);
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Sort objects by the following three criteria.
+     *
+     * <li>
+     *     The number executors of the topology that needs to be scheduled is already on the object (node or rack)
+     *     in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on
+     *     the same object (node or rack) as the existing executors of the topology.
+     * </li>
+     *
+     * <li>
+     *     The subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     *     resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     *     resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     *     How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     *     requested by the executor that the sorting is being done for and pulls it down if it is not.
+     *     By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     *     above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     *     resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     *     have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.
+     *     This is similar to logic used {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     * </li>
+     *
+     * <li>
+     *     The tie between two nodes with same resource availability is broken by using the node with lower minimum
+     *     percentage used. This comparison was used in {@link #sortObjectResourcesDefault(ObjectResourcesSummary, ExistingScheduleFunc)}
+     *     but here it is made subservient to modified resource availbility used in
+     *     {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     *
+     * </li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    private Iterable<ObjectResourcesItem> sortObjectResourcesCommon(
+            final ObjectResourcesSummary allResources, final ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        // Copy and modify allResources
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        final NormalizedResourceRequest requestedResources = (exec != null) ? topologyDetails.getTotalResources(exec) : null;
+        affinityBasedAllResources.getObjectResources().forEach(
+            x -> {
+                if (requestedResources != null) {
+                    // negate unrequested resources
+                    x.availableResources.updateForRareResourceAffinity(requestedResources);
+                }
+                x.minResourcePercent = availableResourcesOverall.calculateMinPercentageUsedBy(x.availableResources);
+                x.avgResourcePercent = availableResourcesOverall.calculateAveragePercentageUsedBy(x.availableResources);
+
+                LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                        x.id, x.minResourcePercent, x.avgResourcePercent,
+                        existingScheduleFunc.getNumExistingSchedule(x.id));
+            }
+        );
+
+        // Use the following comparator to sort
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double o1Avg = o1.avgResourcePercent;
+            double o2Avg = o2.avgResourcePercent;
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     * requested by the executor that the sorting is being done for and pulls it down if it is not.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     * resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     * have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesGeneric(
+            final ObjectResourcesSummary allResources, ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        affinityBasedAllResources.getObjectResources()
+                .forEach(x -> x.availableResources.updateForRareResourceAffinity(requestedResources));
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            double o1Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o1.availableResources);
+            double o2Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o2.availableResources);
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability. So we will be less likely to pick
+     * a rack that have a lot of one resource but a low amount of another.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesDefault(
+            final ObjectResourcesSummary allResources,
+            final ExistingScheduleFunc existingScheduleFunc) {
+
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        for (ObjectResourcesItem objectResources : allResources.getObjectResources()) {
+            objectResources.minResourcePercent =
+                    availableResourcesOverall.calculateMinPercentageUsedBy(objectResources.availableResources);
+            objectResources.avgResourcePercent =
+                    availableResourcesOverall.calculateAveragePercentageUsedBy(objectResources.availableResources);
+            LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                    objectResources.id, objectResources.minResourcePercent, objectResources.avgResourcePercent,
+                    existingScheduleFunc.getNumExistingSchedule(objectResources.id));
+        }
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double diff = o1.avgResourcePercent - o2.avgResourcePercent;
+            if (diff > 0.0) {
+                return -1;
+            } else if (diff < 0.0) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(allResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availHosts a collection of all the hosts we want to sort
+     * @param rackId     the rack id availNodes are a part of
+     * @return an iterable of sorted hosts.
+     */
+    private Iterable<ObjectResourcesItem> sortHosts(
+            Collection<String> availHosts, ExecutorDetails exec, String rackId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary rackResourcesSummary = new ObjectResourcesSummary("RACK");
+        availHosts.forEach(h -> {
+            ObjectResourcesItem hostItem = new ObjectResourcesItem(h);
+            for (RasNode x : hostnameToNodes.get(h)) {
+                hostItem.add(new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0));
+            }
+            rackResourcesSummary.addObjectResourcesItem(hostItem);
+        });
+
+        LOG.debug(
+                "Rack {}: Overall Avail [ {} ] Total [ {} ]",
+                rackId,
+                rackResourcesSummary.getAvailableResourcesOverall(),
+                rackResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            rackResourcesSummary,
+            exec,
+            (hostId) -> {
+                AtomicInteger count = scheduledCount.get(hostId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availRasNodes a list of all the nodes we want to sort
+     * @param hostId     the host-id that availNodes are a part of
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem} for nodes.
+     */
+    private Iterable<ObjectResourcesItem> sortNodes(
+            List<RasNode> availRasNodes, ExecutorDetails exec, String hostId,
+            Map<String, AtomicInteger> scheduledCount) {

Review comment:
       Easier to handle incrementing with a mutable Integer: otherwise a single statement becomes a three liner  with immutable Integer - get, increment, set.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582104768



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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;
+    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 (hostName == null) {
+                // ignore supervisors on blacklistedHosts
+                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 = computeRackToHosts(cluster, superIdToRack);
+
+        // 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;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        Set<String> retVal = new HashSet<>();
+        requestedResources.toNormalizedMap().entrySet().forEach(
+            e -> {
+                if (e.getValue() > 0.0) {
+                    retVal.add(e.getKey());
+                }
+            });
+        return retVal;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.
+     *
+     * @param resourcesSummary     contains all individual {@link ObjectResourcesItem} as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    protected Iterable<ObjectResourcesItem> sortObjectResources(
+            ObjectResourcesSummary resourcesSummary, ExecutorDetails exec, ExistingScheduleFunc existingScheduleFunc) {
+        switch (nodeSortType) {
+            case DEFAULT_RAS:
+                return sortObjectResourcesDefault(resourcesSummary, existingScheduleFunc);
+            case GENERIC_RAS:
+                return sortObjectResourcesGeneric(resourcesSummary, exec, existingScheduleFunc);
+            case COMMON:
+                return sortObjectResourcesCommon(resourcesSummary, exec, existingScheduleFunc);
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Sort objects by the following three criteria.
+     *
+     * <li>
+     *     The number executors of the topology that needs to be scheduled is already on the object (node or rack)
+     *     in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on
+     *     the same object (node or rack) as the existing executors of the topology.
+     * </li>
+     *
+     * <li>
+     *     The subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     *     resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     *     resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     *     How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     *     requested by the executor that the sorting is being done for and pulls it down if it is not.
+     *     By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     *     above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     *     resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     *     have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.
+     *     This is similar to logic used {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     * </li>
+     *
+     * <li>
+     *     The tie between two nodes with same resource availability is broken by using the node with lower minimum
+     *     percentage used. This comparison was used in {@link #sortObjectResourcesDefault(ObjectResourcesSummary, ExistingScheduleFunc)}
+     *     but here it is made subservient to modified resource availbility used in
+     *     {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     *
+     * </li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    private Iterable<ObjectResourcesItem> sortObjectResourcesCommon(
+            final ObjectResourcesSummary allResources, final ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        // Copy and modify allResources
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        final NormalizedResourceRequest requestedResources = (exec != null) ? topologyDetails.getTotalResources(exec) : null;
+        affinityBasedAllResources.getObjectResources().forEach(
+            x -> {
+                if (requestedResources != null) {
+                    // negate unrequested resources
+                    x.availableResources.updateForRareResourceAffinity(requestedResources);
+                }
+                x.minResourcePercent = availableResourcesOverall.calculateMinPercentageUsedBy(x.availableResources);
+                x.avgResourcePercent = availableResourcesOverall.calculateAveragePercentageUsedBy(x.availableResources);
+
+                LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                        x.id, x.minResourcePercent, x.avgResourcePercent,
+                        existingScheduleFunc.getNumExistingSchedule(x.id));
+            }
+        );
+
+        // Use the following comparator to sort
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double o1Avg = o1.avgResourcePercent;
+            double o2Avg = o2.avgResourcePercent;
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     * requested by the executor that the sorting is being done for and pulls it down if it is not.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     * resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     * have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesGeneric(
+            final ObjectResourcesSummary allResources, ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        affinityBasedAllResources.getObjectResources()
+                .forEach(x -> x.availableResources.updateForRareResourceAffinity(requestedResources));
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            double o1Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o1.availableResources);
+            double o2Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o2.availableResources);
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability. So we will be less likely to pick
+     * a rack that have a lot of one resource but a low amount of another.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesDefault(
+            final ObjectResourcesSummary allResources,
+            final ExistingScheduleFunc existingScheduleFunc) {
+
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        for (ObjectResourcesItem objectResources : allResources.getObjectResources()) {
+            objectResources.minResourcePercent =
+                    availableResourcesOverall.calculateMinPercentageUsedBy(objectResources.availableResources);
+            objectResources.avgResourcePercent =
+                    availableResourcesOverall.calculateAveragePercentageUsedBy(objectResources.availableResources);
+            LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                    objectResources.id, objectResources.minResourcePercent, objectResources.avgResourcePercent,
+                    existingScheduleFunc.getNumExistingSchedule(objectResources.id));
+        }
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double diff = o1.avgResourcePercent - o2.avgResourcePercent;
+            if (diff > 0.0) {
+                return -1;
+            } else if (diff < 0.0) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(allResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availHosts a collection of all the hosts we want to sort
+     * @param rackId     the rack id availNodes are a part of
+     * @return an iterable of sorted hosts.
+     */
+    private Iterable<ObjectResourcesItem> sortHosts(
+            Collection<String> availHosts, ExecutorDetails exec, String rackId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary rackResourcesSummary = new ObjectResourcesSummary("RACK");
+        availHosts.forEach(h -> {
+            ObjectResourcesItem hostItem = new ObjectResourcesItem(h);
+            for (RasNode x : hostnameToNodes.get(h)) {
+                hostItem.add(new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0));
+            }
+            rackResourcesSummary.addObjectResourcesItem(hostItem);
+        });
+
+        LOG.debug(
+                "Rack {}: Overall Avail [ {} ] Total [ {} ]",
+                rackId,
+                rackResourcesSummary.getAvailableResourcesOverall(),
+                rackResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            rackResourcesSummary,
+            exec,
+            (hostId) -> {
+                AtomicInteger count = scheduledCount.get(hostId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availRasNodes a list of all the nodes we want to sort
+     * @param hostId     the host-id that availNodes are a part of
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem} for nodes.
+     */
+    private Iterable<ObjectResourcesItem> sortNodes(
+            List<RasNode> availRasNodes, ExecutorDetails exec, String hostId,
+            Map<String, AtomicInteger> scheduledCount) {

Review comment:
       It feels like an overuse of AtomicInteger.  If this is only about reducing the code, I would suggest to change it to `Map<String, Integer>` and use getOrDefault method, for example,
   
   ```
   perNodeScheduledCount.computeIfAbsent(superId, (sid) -> new AtomicInteger(0))
                           .getAndAdd(entry.getValue().size());
   ```
   can be changed to 
   
   ```
   perNodeScheduledCount.put(superId, perNodeScheduledCount.getOrDefault(superId, 0) + entry.getValue().size());
   ```
   And 
   ```
   (rackId) -> {
                   AtomicInteger count = scheduledCount.get(rackId);
                   if (count == null) {
                       return 0;
                   }
                   return count.get();
               }
   ```
   can be changed to 
   ```
   (rackId) -> scheduledCount.getOrDefault(rackId, 0);
   ```




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r593423420



##########
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:
       Removed




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r590801811



##########
File path: storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestDefaultResourceAwareStrategy.java
##########
@@ -905,9 +911,9 @@ public void testMultipleRacksWithFavoritism() {
 
         rs.prepareForScheduling(cluster, topo1);
         INodeSorter nodeSorter = new NodeSorter(cluster, topo1, BaseResourceAwareStrategy.NodeSortType.DEFAULT_RAS);

Review comment:
       Do we want to change this to NodeSorterHostProximity too?
   
   or why don't we just rename `NodeSorterHostProximity` to `NodeSorter` and remove the current NodeSorter class? 




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582351464



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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;
+    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 (hostName == null) {
+                // ignore supervisors on blacklistedHosts
+                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 = computeRackToHosts(cluster, superIdToRack);
+
+        // 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;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        Set<String> retVal = new HashSet<>();
+        requestedResources.toNormalizedMap().entrySet().forEach(
+            e -> {
+                if (e.getValue() > 0.0) {
+                    retVal.add(e.getKey());
+                }
+            });
+        return retVal;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.
+     *
+     * @param resourcesSummary     contains all individual {@link ObjectResourcesItem} as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    protected Iterable<ObjectResourcesItem> sortObjectResources(
+            ObjectResourcesSummary resourcesSummary, ExecutorDetails exec, ExistingScheduleFunc existingScheduleFunc) {
+        switch (nodeSortType) {
+            case DEFAULT_RAS:
+                return sortObjectResourcesDefault(resourcesSummary, existingScheduleFunc);
+            case GENERIC_RAS:
+                return sortObjectResourcesGeneric(resourcesSummary, exec, existingScheduleFunc);
+            case COMMON:
+                return sortObjectResourcesCommon(resourcesSummary, exec, existingScheduleFunc);
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Sort objects by the following three criteria.
+     *
+     * <li>
+     *     The number executors of the topology that needs to be scheduled is already on the object (node or rack)
+     *     in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on
+     *     the same object (node or rack) as the existing executors of the topology.
+     * </li>
+     *
+     * <li>
+     *     The subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     *     resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     *     resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     *     How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     *     requested by the executor that the sorting is being done for and pulls it down if it is not.
+     *     By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     *     above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     *     resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     *     have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.
+     *     This is similar to logic used {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     * </li>
+     *
+     * <li>
+     *     The tie between two nodes with same resource availability is broken by using the node with lower minimum
+     *     percentage used. This comparison was used in {@link #sortObjectResourcesDefault(ObjectResourcesSummary, ExistingScheduleFunc)}
+     *     but here it is made subservient to modified resource availbility used in
+     *     {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     *
+     * </li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    private Iterable<ObjectResourcesItem> sortObjectResourcesCommon(
+            final ObjectResourcesSummary allResources, final ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        // Copy and modify allResources
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        final NormalizedResourceRequest requestedResources = (exec != null) ? topologyDetails.getTotalResources(exec) : null;
+        affinityBasedAllResources.getObjectResources().forEach(
+            x -> {
+                if (requestedResources != null) {
+                    // negate unrequested resources
+                    x.availableResources.updateForRareResourceAffinity(requestedResources);
+                }
+                x.minResourcePercent = availableResourcesOverall.calculateMinPercentageUsedBy(x.availableResources);
+                x.avgResourcePercent = availableResourcesOverall.calculateAveragePercentageUsedBy(x.availableResources);
+
+                LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                        x.id, x.minResourcePercent, x.avgResourcePercent,
+                        existingScheduleFunc.getNumExistingSchedule(x.id));
+            }
+        );
+
+        // Use the following comparator to sort
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double o1Avg = o1.avgResourcePercent;
+            double o2Avg = o2.avgResourcePercent;
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     * requested by the executor that the sorting is being done for and pulls it down if it is not.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     * resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     * have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesGeneric(
+            final ObjectResourcesSummary allResources, ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        affinityBasedAllResources.getObjectResources()
+                .forEach(x -> x.availableResources.updateForRareResourceAffinity(requestedResources));
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            double o1Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o1.availableResources);
+            double o2Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o2.availableResources);
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability. So we will be less likely to pick
+     * a rack that have a lot of one resource but a low amount of another.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesDefault(
+            final ObjectResourcesSummary allResources,
+            final ExistingScheduleFunc existingScheduleFunc) {
+
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        for (ObjectResourcesItem objectResources : allResources.getObjectResources()) {
+            objectResources.minResourcePercent =
+                    availableResourcesOverall.calculateMinPercentageUsedBy(objectResources.availableResources);
+            objectResources.avgResourcePercent =
+                    availableResourcesOverall.calculateAveragePercentageUsedBy(objectResources.availableResources);
+            LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                    objectResources.id, objectResources.minResourcePercent, objectResources.avgResourcePercent,
+                    existingScheduleFunc.getNumExistingSchedule(objectResources.id));
+        }
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double diff = o1.avgResourcePercent - o2.avgResourcePercent;
+            if (diff > 0.0) {
+                return -1;
+            } else if (diff < 0.0) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(allResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availHosts a collection of all the hosts we want to sort
+     * @param rackId     the rack id availNodes are a part of
+     * @return an iterable of sorted hosts.
+     */
+    private Iterable<ObjectResourcesItem> sortHosts(
+            Collection<String> availHosts, ExecutorDetails exec, String rackId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary rackResourcesSummary = new ObjectResourcesSummary("RACK");
+        availHosts.forEach(h -> {
+            ObjectResourcesItem hostItem = new ObjectResourcesItem(h);
+            for (RasNode x : hostnameToNodes.get(h)) {
+                hostItem.add(new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0));
+            }
+            rackResourcesSummary.addObjectResourcesItem(hostItem);
+        });
+
+        LOG.debug(
+                "Rack {}: Overall Avail [ {} ] Total [ {} ]",
+                rackId,
+                rackResourcesSummary.getAvailableResourcesOverall(),
+                rackResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            rackResourcesSummary,
+            exec,
+            (hostId) -> {
+                AtomicInteger count = scheduledCount.get(hostId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availRasNodes a list of all the nodes we want to sort
+     * @param hostId     the host-id that availNodes are a part of
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem} for nodes.
+     */
+    private Iterable<ObjectResourcesItem> sortNodes(
+            List<RasNode> availRasNodes, ExecutorDetails exec, String hostId,
+            Map<String, AtomicInteger> scheduledCount) {

Review comment:
       I have seen that use case. First idiom is most common (but ugly) with the put and get smashed into one. Gets you part of the way but not fully like a mutable-int will do. Less common is use of int[1] but some go as far as to create a class. I believe there is a MutableInt class in storm project that is not used anywhere too.
   
   AtomicInteger is the easier and best alternative (in my opinion) for using any mutable integer - because:
      (1) It is mutable
      (2) it is available in Java
   
   I don't understand what you mean by overuse.  I think String and int are probably most overused in Java. 




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582346588



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/BaseResourceAwareStrategy.java
##########
@@ -189,7 +207,7 @@ protected void prepareForScheduling(Cluster cluster, TopologyDetails topologyDet
         LOG.debug("The max state search that will be used by topology {} is {}", topologyDetails.getId(), maxStateSearch);
 
         searcherState = createSearcherState();
-        setNodeSorter(new NodeSorter(cluster, topologyDetails, nodeSortType));
+        setNodeSorter(new NodeSorterHostProximity(cluster, topologyDetails));

Review comment:
       I will create a separate Jira for the code removal.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582087683



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResources.java
##########
@@ -384,14 +391,29 @@ public double calculateMinPercentageUsedBy(NormalizedResources used, double tota
 
     /**
      * If a node or rack has a kind of resource not in a request, make that resource negative so when sorting that node or rack will
-     * be less likely to be selected.
+     * be less likely to be selected. If the resource is in the request, make that resource positive.
      * @param request the requested resources.
      */
     public void updateForRareResourceAffinity(NormalizedResources request) {
         int length = Math.min(this.otherResources.length, request.otherResources.length);
         for (int i = 0; i < length; i++) {
-            if (request.getResourceAt(i) == 0.0) {
-                this.otherResources[i] = -1 * this.otherResources[i];
+            if (request.getResourceAt(i) == 0.0 && this.otherResources[i] > 0.0) {
+                this.otherResources[i] = -this.otherResources[i]; // make negative
+            } else if (request.getResourceAt(i) > 0.0 && this.otherResources[i] < 0.0) {

Review comment:
       Sounds good. But since the object (otherResource) is not reused currently, there is no bug found. Is it correct?




----------------------------------------------------------------
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



[GitHub] [storm] bipinprasad merged pull request #3379: [STORM-3739] Scheduling should sort numa zones by host groups

Posted by GitBox <gi...@apache.org>.
bipinprasad merged pull request #3379:
URL: https://github.com/apache/storm/pull/3379


   


----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582097535



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/BaseResourceAwareStrategy.java
##########
@@ -189,7 +207,7 @@ protected void prepareForScheduling(Cluster cluster, TopologyDetails topologyDet
         LOG.debug("The max state search that will be used by topology {} is {}", topologyDetails.getId(), maxStateSearch);
 
         searcherState = createSearcherState();
-        setNodeSorter(new NodeSorter(cluster, topologyDetails, nodeSortType));
+        setNodeSorter(new NodeSorterHostProximity(cluster, topologyDetails));

Review comment:
       So with this change, whatever scheduler strategy users use, only "common" will be used. 
   
   Basically we are removing the backwards compatibility from https://github.com/apache/storm/blob/master/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/BaseResourceAwareStrategy.java#L61-L62.
   
   Is this change intentional? And if so,  do we want to just remove all of those classes like GenericResourceAwareStrategyOld? (not necessarily a blocker for this PR)
   




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582940769



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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;
+    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 (hostName == null) {
+                // ignore supervisors on blacklistedHosts
+                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 = computeRackToHosts(cluster, superIdToRack);
+
+        // 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;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        Set<String> retVal = new HashSet<>();
+        requestedResources.toNormalizedMap().entrySet().forEach(
+            e -> {
+                if (e.getValue() > 0.0) {
+                    retVal.add(e.getKey());
+                }
+            });
+        return retVal;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.
+     *
+     * @param resourcesSummary     contains all individual {@link ObjectResourcesItem} as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    protected Iterable<ObjectResourcesItem> sortObjectResources(
+            ObjectResourcesSummary resourcesSummary, ExecutorDetails exec, ExistingScheduleFunc existingScheduleFunc) {
+        switch (nodeSortType) {
+            case DEFAULT_RAS:
+                return sortObjectResourcesDefault(resourcesSummary, existingScheduleFunc);
+            case GENERIC_RAS:
+                return sortObjectResourcesGeneric(resourcesSummary, exec, existingScheduleFunc);
+            case COMMON:
+                return sortObjectResourcesCommon(resourcesSummary, exec, existingScheduleFunc);
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Sort objects by the following three criteria.
+     *
+     * <li>
+     *     The number executors of the topology that needs to be scheduled is already on the object (node or rack)
+     *     in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on
+     *     the same object (node or rack) as the existing executors of the topology.
+     * </li>
+     *
+     * <li>
+     *     The subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     *     resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     *     resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     *     How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     *     requested by the executor that the sorting is being done for and pulls it down if it is not.
+     *     By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     *     above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     *     resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     *     have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.
+     *     This is similar to logic used {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     * </li>
+     *
+     * <li>
+     *     The tie between two nodes with same resource availability is broken by using the node with lower minimum
+     *     percentage used. This comparison was used in {@link #sortObjectResourcesDefault(ObjectResourcesSummary, ExistingScheduleFunc)}
+     *     but here it is made subservient to modified resource availbility used in
+     *     {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     *
+     * </li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    private Iterable<ObjectResourcesItem> sortObjectResourcesCommon(
+            final ObjectResourcesSummary allResources, final ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        // Copy and modify allResources
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        final NormalizedResourceRequest requestedResources = (exec != null) ? topologyDetails.getTotalResources(exec) : null;
+        affinityBasedAllResources.getObjectResources().forEach(
+            x -> {
+                if (requestedResources != null) {
+                    // negate unrequested resources
+                    x.availableResources.updateForRareResourceAffinity(requestedResources);
+                }
+                x.minResourcePercent = availableResourcesOverall.calculateMinPercentageUsedBy(x.availableResources);
+                x.avgResourcePercent = availableResourcesOverall.calculateAveragePercentageUsedBy(x.availableResources);
+
+                LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                        x.id, x.minResourcePercent, x.avgResourcePercent,
+                        existingScheduleFunc.getNumExistingSchedule(x.id));
+            }
+        );
+
+        // Use the following comparator to sort
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double o1Avg = o1.avgResourcePercent;
+            double o2Avg = o2.avgResourcePercent;
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     * requested by the executor that the sorting is being done for and pulls it down if it is not.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     * resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     * have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesGeneric(
+            final ObjectResourcesSummary allResources, ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        affinityBasedAllResources.getObjectResources()
+                .forEach(x -> x.availableResources.updateForRareResourceAffinity(requestedResources));
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            double o1Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o1.availableResources);
+            double o2Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o2.availableResources);
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability. So we will be less likely to pick
+     * a rack that have a lot of one resource but a low amount of another.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesDefault(
+            final ObjectResourcesSummary allResources,
+            final ExistingScheduleFunc existingScheduleFunc) {
+
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        for (ObjectResourcesItem objectResources : allResources.getObjectResources()) {
+            objectResources.minResourcePercent =
+                    availableResourcesOverall.calculateMinPercentageUsedBy(objectResources.availableResources);
+            objectResources.avgResourcePercent =
+                    availableResourcesOverall.calculateAveragePercentageUsedBy(objectResources.availableResources);
+            LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                    objectResources.id, objectResources.minResourcePercent, objectResources.avgResourcePercent,
+                    existingScheduleFunc.getNumExistingSchedule(objectResources.id));
+        }
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double diff = o1.avgResourcePercent - o2.avgResourcePercent;
+            if (diff > 0.0) {
+                return -1;
+            } else if (diff < 0.0) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(allResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availHosts a collection of all the hosts we want to sort
+     * @param rackId     the rack id availNodes are a part of
+     * @return an iterable of sorted hosts.
+     */
+    private Iterable<ObjectResourcesItem> sortHosts(
+            Collection<String> availHosts, ExecutorDetails exec, String rackId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary rackResourcesSummary = new ObjectResourcesSummary("RACK");
+        availHosts.forEach(h -> {
+            ObjectResourcesItem hostItem = new ObjectResourcesItem(h);
+            for (RasNode x : hostnameToNodes.get(h)) {
+                hostItem.add(new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0));
+            }
+            rackResourcesSummary.addObjectResourcesItem(hostItem);
+        });
+
+        LOG.debug(
+                "Rack {}: Overall Avail [ {} ] Total [ {} ]",
+                rackId,
+                rackResourcesSummary.getAvailableResourcesOverall(),
+                rackResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            rackResourcesSummary,
+            exec,
+            (hostId) -> {
+                AtomicInteger count = scheduledCount.get(hostId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availRasNodes a list of all the nodes we want to sort
+     * @param hostId     the host-id that availNodes are a part of
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem} for nodes.
+     */
+    private Iterable<ObjectResourcesItem> sortNodes(
+            List<RasNode> availRasNodes, ExecutorDetails exec, String hostId,
+            Map<String, AtomicInteger> scheduledCount) {

Review comment:
       I would still suggest against the use of AtomicInteger here. 
   
   If you think the first implementation is ugly, you can separate them to two lines.
   ```
   int curCnt = perNodeScheduledCount.getOrDefault(superId, 0);
   perNodeScheduledCount.put(superId, curCnt + entry.getValue().size());
   ```
   `perNodeScheduledCount` is a map and increasing the value of the map is a normal practice. 
   
   And the second implementation also looks simpler comparing to AtomicInteger.
   
   What I meant by overuse of AtomicInteger is when people (at least me) look at AtomicInteger, they wonder if there is concurrency going on and they have to think twice before making any changes to it.
   
   AtomicInteger has int value inside the object (if you think int is overused)




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r584026391



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorter.java
##########
@@ -93,21 +96,15 @@ public NodeSorter(Cluster cluster, TopologyDetails topologyDetails, BaseResource
 
         // from Cluster
         networkTopography = cluster.getNetworkTopography();
-        Map<String, String> hostToRack = new HashMap<>();
-        for (Map.Entry<String, List<String>> entry : networkTopography.entrySet()) {
-            String rackId = entry.getKey();
-            for (String hostName: entry.getValue()) {
-                hostToRack.put(hostName, rackId);
-            }
-        }
+        Map<String, String> hostToRack = cluster.getHostToRack();
         RasNodes nodes = new RasNodes(cluster);
         for (RasNode node: nodes.getNodes()) {
             String superId = node.getId();
             String hostName = node.getHostname();

Review comment:
       I probably should, just for the sake of consistency even though this class will be removed later.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r580773605



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,717 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.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, List<RasNode>> rackIdToNodes = new HashMap<>();
+    private final Map<String, List<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();
+        Map<String, String> hostToRack = cluster.getHostToRack();
+        RasNodes nodes = new RasNodes(cluster);
+        for (RasNode node: nodes.getNodes()) {
+            String superId = node.getId();
+            String hostName = node.getHostname();
+            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, id -> new ArrayList<>()).add(hostName);
+        }
+        this.greyListedSupervisorIds = cluster.getGreyListedSupervisors();
+
+        // 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);
+    }
+
+    @Override
+    public void prepare(ExecutorDetails exec) {
+        this.exec = exec;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        Set<String> retVal = new HashSet<>();
+        requestedResources.toNormalizedMap().entrySet().forEach(
+            e -> {
+                if (e.getValue() > 0.0) {
+                    retVal.add(e.getKey());
+                }
+            });
+        return retVal;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.
+     *
+     * @param resourcesSummary     contains all individual {@link ObjectResourcesItem} as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    protected Iterable<ObjectResourcesItem> sortObjectResources(
+            ObjectResourcesSummary resourcesSummary, ExecutorDetails exec, ExistingScheduleFunc existingScheduleFunc) {
+        switch (nodeSortType) {
+            case DEFAULT_RAS:
+                return sortObjectResourcesDefault(resourcesSummary, existingScheduleFunc);
+            case GENERIC_RAS:
+                return sortObjectResourcesGeneric(resourcesSummary, exec, existingScheduleFunc);
+            case COMMON:
+                return sortObjectResourcesCommon(resourcesSummary, exec, existingScheduleFunc);
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Sort objects by the following three criteria.
+     *
+     * <li>
+     *     The number executors of the topology that needs to be scheduled is already on the object (node or rack)
+     *     in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on
+     *     the same object (node or rack) as the existing executors of the topology.
+     * </li>
+     *
+     * <li>
+     *     The subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     *     resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     *     resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     *     How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     *     requested by the executor that the sorting is being done for and pulls it down if it is not.
+     *     By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     *     above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     *     resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     *     have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.
+     *     This is similar to logic used {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     * </li>
+     *
+     * <li>
+     *     The tie between two nodes with same resource availability is broken by using the node with lower minimum
+     *     percentage used. This comparison was used in {@link #sortObjectResourcesDefault(ObjectResourcesSummary, ExistingScheduleFunc)}
+     *     but here it is made subservient to modified resource availbility used in
+     *     {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     *
+     * </li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    private Iterable<ObjectResourcesItem> sortObjectResourcesCommon(
+            final ObjectResourcesSummary allResources, final ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        // Copy and modify allResources
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        final NormalizedResourceRequest requestedResources = (exec != null) ? topologyDetails.getTotalResources(exec) : null;
+        affinityBasedAllResources.getObjectResources().forEach(
+            x -> {
+                if (requestedResources != null) {
+                    // negate unrequested resources
+                    x.availableResources.updateForRareResourceAffinity(requestedResources);
+                }
+                x.minResourcePercent = availableResourcesOverall.calculateMinPercentageUsedBy(x.availableResources);
+                x.avgResourcePercent = availableResourcesOverall.calculateAveragePercentageUsedBy(x.availableResources);
+
+                LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                        x.id, x.minResourcePercent, x.avgResourcePercent,
+                        existingScheduleFunc.getNumExistingSchedule(x.id));
+            }
+        );
+
+        // Use the following comparator to sort
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double o1Avg = o1.avgResourcePercent;
+            double o2Avg = o2.avgResourcePercent;
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     * requested by the executor that the sorting is being done for and pulls it down if it is not.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     * resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     * have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesGeneric(
+            final ObjectResourcesSummary allResources, ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        affinityBasedAllResources.getObjectResources()
+                .forEach(x -> x.availableResources.updateForRareResourceAffinity(requestedResources));
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            double o1Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o1.availableResources);
+            double o2Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o2.availableResources);
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability. So we will be less likely to pick
+     * a rack that have a lot of one resource but a low amount of another.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesDefault(
+            final ObjectResourcesSummary allResources,
+            final ExistingScheduleFunc existingScheduleFunc) {
+
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        for (ObjectResourcesItem objectResources : allResources.getObjectResources()) {
+            objectResources.minResourcePercent =
+                    availableResourcesOverall.calculateMinPercentageUsedBy(objectResources.availableResources);
+            objectResources.avgResourcePercent =
+                    availableResourcesOverall.calculateAveragePercentageUsedBy(objectResources.availableResources);
+            LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                    objectResources.id, objectResources.minResourcePercent, objectResources.avgResourcePercent,
+                    existingScheduleFunc.getNumExistingSchedule(objectResources.id));
+        }
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double diff = o1.avgResourcePercent - o2.avgResourcePercent;
+            if (diff > 0.0) {
+                return -1;
+            } else if (diff < 0.0) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(allResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availHosts a list of all the hosts we want to sort
+     * @param rackId     the rack id availNodes are a part of
+     * @return an iterable of sorted hosts.
+     */
+    private Iterable<ObjectResourcesItem> sortHosts(
+            List<String> availHosts, ExecutorDetails exec, String rackId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary rackResourcesSummary = new ObjectResourcesSummary("RACK");
+        availHosts.forEach(h -> {
+            ObjectResourcesItem hostItem = new ObjectResourcesItem(h);
+            for (RasNode x : hostnameToNodes.get(h)) {
+                hostItem.add(new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0));
+            }
+            rackResourcesSummary.addObjectResourcesItem(hostItem);
+        });
+
+        LOG.debug(
+                "Rack {}: Overall Avail [ {} ] Total [ {} ]",
+                rackId,
+                rackResourcesSummary.getAvailableResourcesOverall(),
+                rackResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            rackResourcesSummary,
+            exec,
+            (hostId) -> {
+                AtomicInteger count = scheduledCount.get(hostId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availRasNodes a list of all the nodes we want to sort
+     * @param hostId     the host-id that availNodes are a part of
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem} for nodes.
+     */
+    private Iterable<ObjectResourcesItem> sortNodes(
+            List<RasNode> availRasNodes, ExecutorDetails exec, String hostId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary hostResourcesSummary = new ObjectResourcesSummary("HOST");
+        availRasNodes.forEach(x ->
+                hostResourcesSummary.addObjectResourcesItem(
+                        new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0)
+                )
+        );
+
+        LOG.debug(
+            "Host {}: Overall Avail [ {} ] Total [ {} ]",
+            hostId,
+            hostResourcesSummary.getAvailableResourcesOverall(),
+            hostResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            hostResourcesSummary,
+            exec,
+            (superId) -> {
+                AtomicInteger count = scheduledCount.get(superId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    protected List<String> makeHostToNodeIds(List<String> hosts) {
+        if (hosts == null) {
+            return Collections.emptyList();
+        }
+        List<String> ret = new ArrayList<>(hosts.size());
+        for (String host: hosts) {
+            List<RasNode> nodes = hostnameToNodes.get(host);
+            if (nodes != null) {
+                for (RasNode node : nodes) {
+                    ret.add(node.getId());
+                }
+            }
+        }
+        return ret;
+    }
+
+    private class LazyNodeSortingIterator implements Iterator<String> {
+        private final LazyNodeSorting parent;
+        private final Iterator<ObjectResourcesItem> rackIterator;
+        private Iterator<ObjectResourcesItem> hostIterator;
+        private Iterator<ObjectResourcesItem> nodeIterator;
+        private String nextValueFromNode = null;
+        private final Iterator<String> pre;
+        private final Iterator<String> post;
+        private final Set<String> skip;
+
+        LazyNodeSortingIterator(LazyNodeSorting parent, Iterable<ObjectResourcesItem> sortedRacks) {
+            this.parent = parent;
+            rackIterator = sortedRacks.iterator();
+            pre = favoredNodeIds.iterator();
+            post = Stream.concat(unFavoredNodeIds.stream(), greyListedSupervisorIds.stream())
+                            .collect(Collectors.toList())
+                            .iterator();
+            skip = parent.skippedNodeIds;
+        }
+
+        private Iterator<ObjectResourcesItem> getNodeIterator() {
+            if (nodeIterator != null && nodeIterator.hasNext()) {
+                return nodeIterator;
+            }
+            //need to get the next host/node iterator
+            if (hostIterator != null && hostIterator.hasNext()) {
+                ObjectResourcesItem host = hostIterator.next();
+                final String hostId = host.id;
+                nodeIterator = parent.getSortedNodesForHost(hostId).iterator();
+                return nodeIterator;
+            }
+            if (rackIterator.hasNext()) {
+                ObjectResourcesItem rack = rackIterator.next();
+                final String rackId = rack.id;
+                hostIterator = parent.getSortedHostsForRack(rackId).iterator();
+                ObjectResourcesItem host = hostIterator.next();
+                final String hostId = host.id;
+                nodeIterator = parent.getSortedNodesForHost(hostId).iterator();
+                return nodeIterator;
+            }
+
+            return null;
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (pre.hasNext()) {
+                return true;
+            }
+            if (nextValueFromNode != null) {
+                return true;
+            }
+            while (true) {
+                //For the node we don't know if we have another one unless we look at the contents
+                Iterator<ObjectResourcesItem> nodeIterator = getNodeIterator();
+                if (nodeIterator == null || !nodeIterator.hasNext()) {
+                    break;
+                }
+                String tmp = nodeIterator.next().id;
+                if (!skip.contains(tmp)) {
+                    nextValueFromNode = tmp;
+                    return true;
+                }
+            }
+            return post.hasNext();
+        }
+
+        @Override
+        public String next() {
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            if (pre.hasNext()) {
+                return pre.next();
+            }
+            if (nextValueFromNode != null) {
+                String tmp = nextValueFromNode;
+                nextValueFromNode = null;
+                return tmp;
+            }
+            return post.next();
+        }
+    }
+
+    private class LazyNodeSorting implements Iterable<String> {
+        private final Map<String, AtomicInteger> perHostScheduledCount = new HashMap<>();
+        private final Map<String, AtomicInteger> perNodeScheduledCount = new HashMap<>();
+        private final Iterable<ObjectResourcesItem> sortedRacks;
+        private final Map<String, Iterable<ObjectResourcesItem>> cachedHosts = new HashMap<>();
+        private final Map<String, Iterable<ObjectResourcesItem>> cachedNodesByHost = new HashMap<>();
+        private final ExecutorDetails exec;
+        private final Set<String> skippedNodeIds = new HashSet<>();
+
+        LazyNodeSorting(ExecutorDetails exec) {
+            this.exec = exec;
+            skippedNodeIds.addAll(favoredNodeIds);
+            skippedNodeIds.addAll(unFavoredNodeIds);
+            skippedNodeIds.addAll(greyListedSupervisorIds);
+
+            String topoId = topologyDetails.getId();
+            SchedulerAssignment assignment = cluster.getAssignmentById(topoId);
+            if (assignment != null) {
+                for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> entry :
+                    assignment.getSlotToExecutors().entrySet()) {
+                    String superId = entry.getKey().getNodeId();
+                    String hostId = nodeIdToHostname.get(superId);
+                    perHostScheduledCount.computeIfAbsent(hostId, id -> new AtomicInteger(0))
+                        .getAndAdd(entry.getValue().size());
+                    perNodeScheduledCount.computeIfAbsent(superId, id -> new AtomicInteger(0))
+                        .getAndAdd(entry.getValue().size());
+                }
+            }
+            sortedRacks = getSortedRacks();
+        }
+
+        private Iterable<ObjectResourcesItem> getSortedHostsForRack(String rackId) {
+            return cachedHosts.computeIfAbsent(rackId,
+                id -> sortHosts(rackIdToHosts.getOrDefault(id, Collections.emptyList()), exec, id, perHostScheduledCount));
+        }
+
+        private Iterable<ObjectResourcesItem> getSortedNodesForHost(String hostId) {
+            return cachedNodesByHost.computeIfAbsent(hostId,
+                id -> sortNodes(hostnameToNodes.getOrDefault(id, Collections.emptyList()), exec, id, perNodeScheduledCount));
+        }
+
+        @Override
+        public Iterator<String> iterator() {
+            return new LazyNodeSortingIterator(this, sortedRacks);
+        }
+    }
+
+    @Override
+    public Iterable<String> sortAllNodes() {
+        //LOG.info("sortAllNodes():cachedLazyNodeIterators.size={}, execRequestResourceKey={}",

Review comment:
       removed




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582940769



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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;
+    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 (hostName == null) {
+                // ignore supervisors on blacklistedHosts
+                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 = computeRackToHosts(cluster, superIdToRack);
+
+        // 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;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        Set<String> retVal = new HashSet<>();
+        requestedResources.toNormalizedMap().entrySet().forEach(
+            e -> {
+                if (e.getValue() > 0.0) {
+                    retVal.add(e.getKey());
+                }
+            });
+        return retVal;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.
+     *
+     * @param resourcesSummary     contains all individual {@link ObjectResourcesItem} as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    protected Iterable<ObjectResourcesItem> sortObjectResources(
+            ObjectResourcesSummary resourcesSummary, ExecutorDetails exec, ExistingScheduleFunc existingScheduleFunc) {
+        switch (nodeSortType) {
+            case DEFAULT_RAS:
+                return sortObjectResourcesDefault(resourcesSummary, existingScheduleFunc);
+            case GENERIC_RAS:
+                return sortObjectResourcesGeneric(resourcesSummary, exec, existingScheduleFunc);
+            case COMMON:
+                return sortObjectResourcesCommon(resourcesSummary, exec, existingScheduleFunc);
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Sort objects by the following three criteria.
+     *
+     * <li>
+     *     The number executors of the topology that needs to be scheduled is already on the object (node or rack)
+     *     in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on
+     *     the same object (node or rack) as the existing executors of the topology.
+     * </li>
+     *
+     * <li>
+     *     The subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     *     resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     *     resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     *     How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     *     requested by the executor that the sorting is being done for and pulls it down if it is not.
+     *     By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     *     above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     *     resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     *     have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.
+     *     This is similar to logic used {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     * </li>
+     *
+     * <li>
+     *     The tie between two nodes with same resource availability is broken by using the node with lower minimum
+     *     percentage used. This comparison was used in {@link #sortObjectResourcesDefault(ObjectResourcesSummary, ExistingScheduleFunc)}
+     *     but here it is made subservient to modified resource availbility used in
+     *     {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     *
+     * </li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    private Iterable<ObjectResourcesItem> sortObjectResourcesCommon(
+            final ObjectResourcesSummary allResources, final ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        // Copy and modify allResources
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        final NormalizedResourceRequest requestedResources = (exec != null) ? topologyDetails.getTotalResources(exec) : null;
+        affinityBasedAllResources.getObjectResources().forEach(
+            x -> {
+                if (requestedResources != null) {
+                    // negate unrequested resources
+                    x.availableResources.updateForRareResourceAffinity(requestedResources);
+                }
+                x.minResourcePercent = availableResourcesOverall.calculateMinPercentageUsedBy(x.availableResources);
+                x.avgResourcePercent = availableResourcesOverall.calculateAveragePercentageUsedBy(x.availableResources);
+
+                LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                        x.id, x.minResourcePercent, x.avgResourcePercent,
+                        existingScheduleFunc.getNumExistingSchedule(x.id));
+            }
+        );
+
+        // Use the following comparator to sort
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double o1Avg = o1.avgResourcePercent;
+            double o2Avg = o2.avgResourcePercent;
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     * requested by the executor that the sorting is being done for and pulls it down if it is not.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     * resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     * have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesGeneric(
+            final ObjectResourcesSummary allResources, ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        affinityBasedAllResources.getObjectResources()
+                .forEach(x -> x.availableResources.updateForRareResourceAffinity(requestedResources));
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            double o1Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o1.availableResources);
+            double o2Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o2.availableResources);
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability. So we will be less likely to pick
+     * a rack that have a lot of one resource but a low amount of another.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesDefault(
+            final ObjectResourcesSummary allResources,
+            final ExistingScheduleFunc existingScheduleFunc) {
+
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        for (ObjectResourcesItem objectResources : allResources.getObjectResources()) {
+            objectResources.minResourcePercent =
+                    availableResourcesOverall.calculateMinPercentageUsedBy(objectResources.availableResources);
+            objectResources.avgResourcePercent =
+                    availableResourcesOverall.calculateAveragePercentageUsedBy(objectResources.availableResources);
+            LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                    objectResources.id, objectResources.minResourcePercent, objectResources.avgResourcePercent,
+                    existingScheduleFunc.getNumExistingSchedule(objectResources.id));
+        }
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double diff = o1.avgResourcePercent - o2.avgResourcePercent;
+            if (diff > 0.0) {
+                return -1;
+            } else if (diff < 0.0) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(allResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availHosts a collection of all the hosts we want to sort
+     * @param rackId     the rack id availNodes are a part of
+     * @return an iterable of sorted hosts.
+     */
+    private Iterable<ObjectResourcesItem> sortHosts(
+            Collection<String> availHosts, ExecutorDetails exec, String rackId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary rackResourcesSummary = new ObjectResourcesSummary("RACK");
+        availHosts.forEach(h -> {
+            ObjectResourcesItem hostItem = new ObjectResourcesItem(h);
+            for (RasNode x : hostnameToNodes.get(h)) {
+                hostItem.add(new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0));
+            }
+            rackResourcesSummary.addObjectResourcesItem(hostItem);
+        });
+
+        LOG.debug(
+                "Rack {}: Overall Avail [ {} ] Total [ {} ]",
+                rackId,
+                rackResourcesSummary.getAvailableResourcesOverall(),
+                rackResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            rackResourcesSummary,
+            exec,
+            (hostId) -> {
+                AtomicInteger count = scheduledCount.get(hostId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availRasNodes a list of all the nodes we want to sort
+     * @param hostId     the host-id that availNodes are a part of
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem} for nodes.
+     */
+    private Iterable<ObjectResourcesItem> sortNodes(
+            List<RasNode> availRasNodes, ExecutorDetails exec, String hostId,
+            Map<String, AtomicInteger> scheduledCount) {

Review comment:
       I would still suggest against the use of AtomicInteger here. 
   
   If you think the first implementation is ugly, you can separate them to two lines.
   ```
   int curCnt = perNodeScheduledCount.getOrDefault(superId, 0);
   perNodeScheduledCount.put(superId, curCnt + entry.getValue().size());
   ```
   `perNodeScheduledCount` is a map and increasing the value of the map is a normal practice. 
   
   And the second implementation also looks simpler comparing to AtomicInteger.
   
   What I meant by overuse of AtomicInteger is when people (at least me) look at AtomicInteger, they wonder if there is concurrency going on and they have to think twice before making any changes to it.
   
   AtomicInteger has int value inside the object (if you since int is overused)




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r581617781



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,717 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.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, List<RasNode>> rackIdToNodes = new HashMap<>();
+    private final Map<String, List<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();
+        Map<String, String> hostToRack = cluster.getHostToRack();
+        RasNodes nodes = new RasNodes(cluster);
+        for (RasNode node: nodes.getNodes()) {
+            String superId = node.getId();
+            String hostName = node.getHostname();
+            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, id -> new ArrayList<>()).add(hostName);
+        }
+        this.greyListedSupervisorIds = cluster.getGreyListedSupervisors();
+
+        // 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);
+    }
+
+    @Override
+    public void prepare(ExecutorDetails exec) {
+        this.exec = exec;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        Set<String> retVal = new HashSet<>();
+        requestedResources.toNormalizedMap().entrySet().forEach(
+            e -> {
+                if (e.getValue() > 0.0) {
+                    retVal.add(e.getKey());
+                }
+            });
+        return retVal;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.
+     *
+     * @param resourcesSummary     contains all individual {@link ObjectResourcesItem} as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    protected Iterable<ObjectResourcesItem> sortObjectResources(
+            ObjectResourcesSummary resourcesSummary, ExecutorDetails exec, ExistingScheduleFunc existingScheduleFunc) {
+        switch (nodeSortType) {
+            case DEFAULT_RAS:
+                return sortObjectResourcesDefault(resourcesSummary, existingScheduleFunc);
+            case GENERIC_RAS:
+                return sortObjectResourcesGeneric(resourcesSummary, exec, existingScheduleFunc);
+            case COMMON:
+                return sortObjectResourcesCommon(resourcesSummary, exec, existingScheduleFunc);
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Sort objects by the following three criteria.
+     *
+     * <li>
+     *     The number executors of the topology that needs to be scheduled is already on the object (node or rack)
+     *     in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on
+     *     the same object (node or rack) as the existing executors of the topology.
+     * </li>
+     *
+     * <li>
+     *     The subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     *     resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     *     resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     *     How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     *     requested by the executor that the sorting is being done for and pulls it down if it is not.
+     *     By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     *     above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     *     resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     *     have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.
+     *     This is similar to logic used {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     * </li>
+     *
+     * <li>
+     *     The tie between two nodes with same resource availability is broken by using the node with lower minimum
+     *     percentage used. This comparison was used in {@link #sortObjectResourcesDefault(ObjectResourcesSummary, ExistingScheduleFunc)}
+     *     but here it is made subservient to modified resource availbility used in
+     *     {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     *
+     * </li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    private Iterable<ObjectResourcesItem> sortObjectResourcesCommon(
+            final ObjectResourcesSummary allResources, final ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        // Copy and modify allResources
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        final NormalizedResourceRequest requestedResources = (exec != null) ? topologyDetails.getTotalResources(exec) : null;
+        affinityBasedAllResources.getObjectResources().forEach(
+            x -> {
+                if (requestedResources != null) {
+                    // negate unrequested resources
+                    x.availableResources.updateForRareResourceAffinity(requestedResources);
+                }
+                x.minResourcePercent = availableResourcesOverall.calculateMinPercentageUsedBy(x.availableResources);
+                x.avgResourcePercent = availableResourcesOverall.calculateAveragePercentageUsedBy(x.availableResources);
+
+                LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                        x.id, x.minResourcePercent, x.avgResourcePercent,
+                        existingScheduleFunc.getNumExistingSchedule(x.id));
+            }
+        );
+
+        // Use the following comparator to sort
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double o1Avg = o1.avgResourcePercent;
+            double o2Avg = o2.avgResourcePercent;
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     * requested by the executor that the sorting is being done for and pulls it down if it is not.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     * resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     * have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesGeneric(
+            final ObjectResourcesSummary allResources, ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        affinityBasedAllResources.getObjectResources()
+                .forEach(x -> x.availableResources.updateForRareResourceAffinity(requestedResources));
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            double o1Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o1.availableResources);
+            double o2Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o2.availableResources);
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability. So we will be less likely to pick
+     * a rack that have a lot of one resource but a low amount of another.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesDefault(
+            final ObjectResourcesSummary allResources,
+            final ExistingScheduleFunc existingScheduleFunc) {
+
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        for (ObjectResourcesItem objectResources : allResources.getObjectResources()) {
+            objectResources.minResourcePercent =
+                    availableResourcesOverall.calculateMinPercentageUsedBy(objectResources.availableResources);
+            objectResources.avgResourcePercent =
+                    availableResourcesOverall.calculateAveragePercentageUsedBy(objectResources.availableResources);
+            LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                    objectResources.id, objectResources.minResourcePercent, objectResources.avgResourcePercent,
+                    existingScheduleFunc.getNumExistingSchedule(objectResources.id));
+        }
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double diff = o1.avgResourcePercent - o2.avgResourcePercent;
+            if (diff > 0.0) {
+                return -1;
+            } else if (diff < 0.0) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(allResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availHosts a list of all the hosts we want to sort
+     * @param rackId     the rack id availNodes are a part of
+     * @return an iterable of sorted hosts.
+     */
+    private Iterable<ObjectResourcesItem> sortHosts(
+            List<String> availHosts, ExecutorDetails exec, String rackId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary rackResourcesSummary = new ObjectResourcesSummary("RACK");
+        availHosts.forEach(h -> {
+            ObjectResourcesItem hostItem = new ObjectResourcesItem(h);
+            for (RasNode x : hostnameToNodes.get(h)) {
+                hostItem.add(new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0));
+            }
+            rackResourcesSummary.addObjectResourcesItem(hostItem);
+        });
+
+        LOG.debug(
+                "Rack {}: Overall Avail [ {} ] Total [ {} ]",
+                rackId,
+                rackResourcesSummary.getAvailableResourcesOverall(),
+                rackResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            rackResourcesSummary,
+            exec,
+            (hostId) -> {
+                AtomicInteger count = scheduledCount.get(hostId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availRasNodes a list of all the nodes we want to sort
+     * @param hostId     the host-id that availNodes are a part of
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem} for nodes.
+     */
+    private Iterable<ObjectResourcesItem> sortNodes(
+            List<RasNode> availRasNodes, ExecutorDetails exec, String hostId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary hostResourcesSummary = new ObjectResourcesSummary("HOST");
+        availRasNodes.forEach(x ->
+                hostResourcesSummary.addObjectResourcesItem(
+                        new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0)
+                )
+        );
+
+        LOG.debug(
+            "Host {}: Overall Avail [ {} ] Total [ {} ]",
+            hostId,
+            hostResourcesSummary.getAvailableResourcesOverall(),
+            hostResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            hostResourcesSummary,
+            exec,
+            (superId) -> {
+                AtomicInteger count = scheduledCount.get(superId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    protected List<String> makeHostToNodeIds(List<String> hosts) {
+        if (hosts == null) {
+            return Collections.emptyList();
+        }
+        List<String> ret = new ArrayList<>(hosts.size());
+        for (String host: hosts) {
+            List<RasNode> nodes = hostnameToNodes.get(host);
+            if (nodes != null) {
+                for (RasNode node : nodes) {
+                    ret.add(node.getId());
+                }
+            }
+        }
+        return ret;
+    }
+
+    private class LazyNodeSortingIterator implements Iterator<String> {
+        private final LazyNodeSorting parent;
+        private final Iterator<ObjectResourcesItem> rackIterator;
+        private Iterator<ObjectResourcesItem> hostIterator;
+        private Iterator<ObjectResourcesItem> nodeIterator;
+        private String nextValueFromNode = null;
+        private final Iterator<String> pre;
+        private final Iterator<String> post;
+        private final Set<String> skip;
+
+        LazyNodeSortingIterator(LazyNodeSorting parent, Iterable<ObjectResourcesItem> sortedRacks) {
+            this.parent = parent;
+            rackIterator = sortedRacks.iterator();
+            pre = favoredNodeIds.iterator();
+            post = Stream.concat(unFavoredNodeIds.stream(), greyListedSupervisorIds.stream())
+                            .collect(Collectors.toList())
+                            .iterator();
+            skip = parent.skippedNodeIds;
+        }
+
+        private Iterator<ObjectResourcesItem> getNodeIterator() {
+            if (nodeIterator != null && nodeIterator.hasNext()) {
+                return nodeIterator;
+            }
+            //need to get the next host/node iterator
+            if (hostIterator != null && hostIterator.hasNext()) {
+                ObjectResourcesItem host = hostIterator.next();
+                final String hostId = host.id;
+                nodeIterator = parent.getSortedNodesForHost(hostId).iterator();
+                return nodeIterator;
+            }
+            if (rackIterator.hasNext()) {
+                ObjectResourcesItem rack = rackIterator.next();
+                final String rackId = rack.id;
+                hostIterator = parent.getSortedHostsForRack(rackId).iterator();
+                ObjectResourcesItem host = hostIterator.next();
+                final String hostId = host.id;
+                nodeIterator = parent.getSortedNodesForHost(hostId).iterator();
+                return nodeIterator;
+            }
+
+            return null;
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (pre.hasNext()) {
+                return true;
+            }
+            if (nextValueFromNode != null) {
+                return true;
+            }
+            while (true) {
+                //For the node we don't know if we have another one unless we look at the contents
+                Iterator<ObjectResourcesItem> nodeIterator = getNodeIterator();
+                if (nodeIterator == null || !nodeIterator.hasNext()) {
+                    break;
+                }
+                String tmp = nodeIterator.next().id;
+                if (!skip.contains(tmp)) {
+                    nextValueFromNode = tmp;
+                    return true;
+                }
+            }
+            return post.hasNext();
+        }
+
+        @Override
+        public String next() {
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            if (pre.hasNext()) {
+                return pre.next();
+            }
+            if (nextValueFromNode != null) {
+                String tmp = nextValueFromNode;
+                nextValueFromNode = null;
+                return tmp;
+            }
+            return post.next();
+        }
+    }
+
+    private class LazyNodeSorting implements Iterable<String> {
+        private final Map<String, AtomicInteger> perHostScheduledCount = new HashMap<>();
+        private final Map<String, AtomicInteger> perNodeScheduledCount = new HashMap<>();
+        private final Iterable<ObjectResourcesItem> sortedRacks;
+        private final Map<String, Iterable<ObjectResourcesItem>> cachedHosts = new HashMap<>();
+        private final Map<String, Iterable<ObjectResourcesItem>> cachedNodesByHost = new HashMap<>();
+        private final ExecutorDetails exec;
+        private final Set<String> skippedNodeIds = new HashSet<>();
+
+        LazyNodeSorting(ExecutorDetails exec) {
+            this.exec = exec;
+            skippedNodeIds.addAll(favoredNodeIds);
+            skippedNodeIds.addAll(unFavoredNodeIds);
+            skippedNodeIds.addAll(greyListedSupervisorIds);
+
+            String topoId = topologyDetails.getId();
+            SchedulerAssignment assignment = cluster.getAssignmentById(topoId);
+            if (assignment != null) {
+                for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> entry :
+                    assignment.getSlotToExecutors().entrySet()) {
+                    String superId = entry.getKey().getNodeId();
+                    String hostId = nodeIdToHostname.get(superId);
+                    perHostScheduledCount.computeIfAbsent(hostId, id -> new AtomicInteger(0))
+                        .getAndAdd(entry.getValue().size());
+                    perNodeScheduledCount.computeIfAbsent(superId, id -> new AtomicInteger(0))
+                        .getAndAdd(entry.getValue().size());
+                }
+            }
+            sortedRacks = getSortedRacks();
+        }
+
+        private Iterable<ObjectResourcesItem> getSortedHostsForRack(String rackId) {
+            return cachedHosts.computeIfAbsent(rackId,
+                id -> sortHosts(rackIdToHosts.getOrDefault(id, Collections.emptyList()), exec, id, perHostScheduledCount));
+        }
+
+        private Iterable<ObjectResourcesItem> getSortedNodesForHost(String hostId) {
+            return cachedNodesByHost.computeIfAbsent(hostId,
+                id -> sortNodes(hostnameToNodes.getOrDefault(id, Collections.emptyList()), exec, id, perNodeScheduledCount));
+        }
+
+        @Override
+        public Iterator<String> iterator() {
+            return new LazyNodeSortingIterator(this, sortedRacks);
+        }
+    }
+
+    @Override
+    public Iterable<String> sortAllNodes() {
+        //LOG.info("sortAllNodes():cachedLazyNodeIterators.size={}, execRequestResourceKey={}",
+        //            cachedLazyNodeIterators.size(), execRequestResourcesKey);
+        //return cachedLazyNodeIterators.computeIfAbsent(execRequestResourcesKey, k -> new LazyNodeSorting(exec));
+        return new LazyNodeSorting(exec);
+    }
+
+    private ObjectResourcesSummary createClusterSummarizedResources() {
+        ObjectResourcesSummary clusterResourcesSummary = new ObjectResourcesSummary("Cluster");
+
+        //This is the first time so initialize the resources.
+        for (Map.Entry<String, List<String>> entry : networkTopography.entrySet()) {
+            String rackId = entry.getKey();
+            List<String> nodeHosts = entry.getValue();
+            ObjectResourcesItem rack = new ObjectResourcesItem(rackId);
+            for (String nodeHost : nodeHosts) {
+                for (RasNode node : hostnameToNodes(nodeHost)) {
+                    rack.availableResources.add(node.getTotalAvailableResources());
+                    rack.totalResources.add(node.getTotalAvailableResources());

Review comment:
       Yes. Fixing.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582841733



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/BaseResourceAwareStrategy.java
##########
@@ -189,7 +207,7 @@ protected void prepareForScheduling(Cluster cluster, TopologyDetails topologyDet
         LOG.debug("The max state search that will be used by topology {} is {}", topologyDetails.getId(), maxStateSearch);
 
         searcherState = createSearcherState();
-        setNodeSorter(new NodeSorter(cluster, topologyDetails, nodeSortType));
+        setNodeSorter(new NodeSorterHostProximity(cluster, topologyDetails));

Review comment:
       https://issues.apache.org/jira/browse/STORM-3747 for removal of deprecated and unused classes.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582335250



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResources.java
##########
@@ -384,14 +391,29 @@ public double calculateMinPercentageUsedBy(NormalizedResources used, double tota
 
     /**
      * If a node or rack has a kind of resource not in a request, make that resource negative so when sorting that node or rack will
-     * be less likely to be selected.
+     * be less likely to be selected. If the resource is in the request, make that resource positive.
      * @param request the requested resources.
      */
     public void updateForRareResourceAffinity(NormalizedResources request) {
         int length = Math.min(this.otherResources.length, request.otherResources.length);
         for (int i = 0; i < length; i++) {
-            if (request.getResourceAt(i) == 0.0) {
-                this.otherResources[i] = -1 * this.otherResources[i];
+            if (request.getResourceAt(i) == 0.0 && this.otherResources[i] > 0.0) {
+                this.otherResources[i] = -this.otherResources[i]; // make negative
+            } else if (request.getResourceAt(i) > 0.0 && this.otherResources[i] < 0.0) {

Review comment:
       But I ran into it when I was trying to optimize the sorting code with reused objects. across executors (you may remember the executor caching key that I removed below getExecRequestKey). I will attempt that speed in a different Jira. But this method needs to work properly and repeatedly with two or more calls.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r581618776



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/ISchedulingState.java
##########
@@ -282,6 +285,18 @@ boolean wouldFit(
      */
     Map<String, List<String>> getNetworkTopography();
 
+    /**
+     * Get host -> rack map - the inverse of networkTopography.
+     */
+    default Map<String, String> getHostToRack() {

Review comment:
       This method won't change based on implementation - since it is just reversing a map provided by other implementation specific methods.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582359287



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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;
+    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 (hostName == null) {
+                // ignore supervisors on blacklistedHosts
+                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 = computeRackToHosts(cluster, superIdToRack);
+
+        // 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;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        Set<String> retVal = new HashSet<>();
+        requestedResources.toNormalizedMap().entrySet().forEach(
+            e -> {
+                if (e.getValue() > 0.0) {
+                    retVal.add(e.getKey());
+                }
+            });
+        return retVal;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.
+     *
+     * @param resourcesSummary     contains all individual {@link ObjectResourcesItem} as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    protected Iterable<ObjectResourcesItem> sortObjectResources(
+            ObjectResourcesSummary resourcesSummary, ExecutorDetails exec, ExistingScheduleFunc existingScheduleFunc) {
+        switch (nodeSortType) {
+            case DEFAULT_RAS:
+                return sortObjectResourcesDefault(resourcesSummary, existingScheduleFunc);
+            case GENERIC_RAS:
+                return sortObjectResourcesGeneric(resourcesSummary, exec, existingScheduleFunc);
+            case COMMON:
+                return sortObjectResourcesCommon(resourcesSummary, exec, existingScheduleFunc);
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Sort objects by the following three criteria.
+     *
+     * <li>
+     *     The number executors of the topology that needs to be scheduled is already on the object (node or rack)
+     *     in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on
+     *     the same object (node or rack) as the existing executors of the topology.
+     * </li>
+     *
+     * <li>
+     *     The subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     *     resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     *     resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     *     How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     *     requested by the executor that the sorting is being done for and pulls it down if it is not.
+     *     By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     *     above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     *     resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     *     have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.
+     *     This is similar to logic used {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     * </li>
+     *
+     * <li>
+     *     The tie between two nodes with same resource availability is broken by using the node with lower minimum
+     *     percentage used. This comparison was used in {@link #sortObjectResourcesDefault(ObjectResourcesSummary, ExistingScheduleFunc)}
+     *     but here it is made subservient to modified resource availbility used in
+     *     {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     *
+     * </li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    private Iterable<ObjectResourcesItem> sortObjectResourcesCommon(
+            final ObjectResourcesSummary allResources, final ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        // Copy and modify allResources
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        final NormalizedResourceRequest requestedResources = (exec != null) ? topologyDetails.getTotalResources(exec) : null;
+        affinityBasedAllResources.getObjectResources().forEach(
+            x -> {
+                if (requestedResources != null) {
+                    // negate unrequested resources
+                    x.availableResources.updateForRareResourceAffinity(requestedResources);
+                }
+                x.minResourcePercent = availableResourcesOverall.calculateMinPercentageUsedBy(x.availableResources);
+                x.avgResourcePercent = availableResourcesOverall.calculateAveragePercentageUsedBy(x.availableResources);
+
+                LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                        x.id, x.minResourcePercent, x.avgResourcePercent,
+                        existingScheduleFunc.getNumExistingSchedule(x.id));
+            }
+        );
+
+        // Use the following comparator to sort
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double o1Avg = o1.avgResourcePercent;
+            double o2Avg = o2.avgResourcePercent;
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     * requested by the executor that the sorting is being done for and pulls it down if it is not.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     * resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     * have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesGeneric(
+            final ObjectResourcesSummary allResources, ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        affinityBasedAllResources.getObjectResources()
+                .forEach(x -> x.availableResources.updateForRareResourceAffinity(requestedResources));
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            double o1Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o1.availableResources);
+            double o2Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o2.availableResources);
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability. So we will be less likely to pick
+     * a rack that have a lot of one resource but a low amount of another.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesDefault(
+            final ObjectResourcesSummary allResources,
+            final ExistingScheduleFunc existingScheduleFunc) {
+
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        for (ObjectResourcesItem objectResources : allResources.getObjectResources()) {
+            objectResources.minResourcePercent =
+                    availableResourcesOverall.calculateMinPercentageUsedBy(objectResources.availableResources);
+            objectResources.avgResourcePercent =
+                    availableResourcesOverall.calculateAveragePercentageUsedBy(objectResources.availableResources);
+            LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                    objectResources.id, objectResources.minResourcePercent, objectResources.avgResourcePercent,
+                    existingScheduleFunc.getNumExistingSchedule(objectResources.id));
+        }
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double diff = o1.avgResourcePercent - o2.avgResourcePercent;
+            if (diff > 0.0) {
+                return -1;
+            } else if (diff < 0.0) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(allResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availHosts a collection of all the hosts we want to sort
+     * @param rackId     the rack id availNodes are a part of
+     * @return an iterable of sorted hosts.
+     */
+    private Iterable<ObjectResourcesItem> sortHosts(
+            Collection<String> availHosts, ExecutorDetails exec, String rackId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary rackResourcesSummary = new ObjectResourcesSummary("RACK");
+        availHosts.forEach(h -> {
+            ObjectResourcesItem hostItem = new ObjectResourcesItem(h);
+            for (RasNode x : hostnameToNodes.get(h)) {
+                hostItem.add(new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0));
+            }
+            rackResourcesSummary.addObjectResourcesItem(hostItem);
+        });
+
+        LOG.debug(
+                "Rack {}: Overall Avail [ {} ] Total [ {} ]",
+                rackId,
+                rackResourcesSummary.getAvailableResourcesOverall(),
+                rackResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            rackResourcesSummary,
+            exec,
+            (hostId) -> {
+                AtomicInteger count = scheduledCount.get(hostId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availRasNodes a list of all the nodes we want to sort
+     * @param hostId     the host-id that availNodes are a part of
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem} for nodes.
+     */
+    private Iterable<ObjectResourcesItem> sortNodes(
+            List<RasNode> availRasNodes, ExecutorDetails exec, String hostId,
+            Map<String, AtomicInteger> scheduledCount) {

Review comment:
       But if you strongly object to use of AtomicInteger let me know. I will revert back to regular int/Integer




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on pull request #3379:
URL: https://github.com/apache/storm/pull/3379#issuecomment-786919760


   And travis build failed 
   ```
   java.lang.AssertionError: topology status TopologyName00047 is not successful Not enough resources to schedule after evicting lower priority topologies. Additional Memory Required: 23040.0 MB (Available: 1401050.0 MB). Additional CPU Required: 175.0% CPU (Available: 25069.0 % CPU).Cannot schedule by GenericResourceAwareStrategy (10001 states traversed in 2083 ms, backtracked 9630 times, 371 of 376 executors scheduled)
   	at org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.assertStatusSuccess(TestUtilsForResourceAwareScheduler.java:516)
   	at org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.assertTopologiesFullyScheduled(TestUtilsForResourceAwareScheduler.java:484)
   	at org.apache.storm.scheduler.resource.strategies.scheduling.TestLargeCluster.testLargeCluster(TestLargeCluster.java:422)
   ```


----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582104768



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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;
+    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 (hostName == null) {
+                // ignore supervisors on blacklistedHosts
+                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 = computeRackToHosts(cluster, superIdToRack);
+
+        // 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;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        Set<String> retVal = new HashSet<>();
+        requestedResources.toNormalizedMap().entrySet().forEach(
+            e -> {
+                if (e.getValue() > 0.0) {
+                    retVal.add(e.getKey());
+                }
+            });
+        return retVal;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.
+     *
+     * @param resourcesSummary     contains all individual {@link ObjectResourcesItem} as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    protected Iterable<ObjectResourcesItem> sortObjectResources(
+            ObjectResourcesSummary resourcesSummary, ExecutorDetails exec, ExistingScheduleFunc existingScheduleFunc) {
+        switch (nodeSortType) {
+            case DEFAULT_RAS:
+                return sortObjectResourcesDefault(resourcesSummary, existingScheduleFunc);
+            case GENERIC_RAS:
+                return sortObjectResourcesGeneric(resourcesSummary, exec, existingScheduleFunc);
+            case COMMON:
+                return sortObjectResourcesCommon(resourcesSummary, exec, existingScheduleFunc);
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Sort objects by the following three criteria.
+     *
+     * <li>
+     *     The number executors of the topology that needs to be scheduled is already on the object (node or rack)
+     *     in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on
+     *     the same object (node or rack) as the existing executors of the topology.
+     * </li>
+     *
+     * <li>
+     *     The subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     *     resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     *     resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     *     How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     *     requested by the executor that the sorting is being done for and pulls it down if it is not.
+     *     By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     *     above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     *     resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     *     have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.
+     *     This is similar to logic used {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     * </li>
+     *
+     * <li>
+     *     The tie between two nodes with same resource availability is broken by using the node with lower minimum
+     *     percentage used. This comparison was used in {@link #sortObjectResourcesDefault(ObjectResourcesSummary, ExistingScheduleFunc)}
+     *     but here it is made subservient to modified resource availbility used in
+     *     {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     *
+     * </li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    private Iterable<ObjectResourcesItem> sortObjectResourcesCommon(
+            final ObjectResourcesSummary allResources, final ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        // Copy and modify allResources
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        final NormalizedResourceRequest requestedResources = (exec != null) ? topologyDetails.getTotalResources(exec) : null;
+        affinityBasedAllResources.getObjectResources().forEach(
+            x -> {
+                if (requestedResources != null) {
+                    // negate unrequested resources
+                    x.availableResources.updateForRareResourceAffinity(requestedResources);
+                }
+                x.minResourcePercent = availableResourcesOverall.calculateMinPercentageUsedBy(x.availableResources);
+                x.avgResourcePercent = availableResourcesOverall.calculateAveragePercentageUsedBy(x.availableResources);
+
+                LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                        x.id, x.minResourcePercent, x.avgResourcePercent,
+                        existingScheduleFunc.getNumExistingSchedule(x.id));
+            }
+        );
+
+        // Use the following comparator to sort
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double o1Avg = o1.avgResourcePercent;
+            double o2Avg = o2.avgResourcePercent;
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     * requested by the executor that the sorting is being done for and pulls it down if it is not.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     * resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     * have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesGeneric(
+            final ObjectResourcesSummary allResources, ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        affinityBasedAllResources.getObjectResources()
+                .forEach(x -> x.availableResources.updateForRareResourceAffinity(requestedResources));
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            double o1Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o1.availableResources);
+            double o2Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o2.availableResources);
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability. So we will be less likely to pick
+     * a rack that have a lot of one resource but a low amount of another.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesDefault(
+            final ObjectResourcesSummary allResources,
+            final ExistingScheduleFunc existingScheduleFunc) {
+
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        for (ObjectResourcesItem objectResources : allResources.getObjectResources()) {
+            objectResources.minResourcePercent =
+                    availableResourcesOverall.calculateMinPercentageUsedBy(objectResources.availableResources);
+            objectResources.avgResourcePercent =
+                    availableResourcesOverall.calculateAveragePercentageUsedBy(objectResources.availableResources);
+            LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                    objectResources.id, objectResources.minResourcePercent, objectResources.avgResourcePercent,
+                    existingScheduleFunc.getNumExistingSchedule(objectResources.id));
+        }
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double diff = o1.avgResourcePercent - o2.avgResourcePercent;
+            if (diff > 0.0) {
+                return -1;
+            } else if (diff < 0.0) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(allResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availHosts a collection of all the hosts we want to sort
+     * @param rackId     the rack id availNodes are a part of
+     * @return an iterable of sorted hosts.
+     */
+    private Iterable<ObjectResourcesItem> sortHosts(
+            Collection<String> availHosts, ExecutorDetails exec, String rackId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary rackResourcesSummary = new ObjectResourcesSummary("RACK");
+        availHosts.forEach(h -> {
+            ObjectResourcesItem hostItem = new ObjectResourcesItem(h);
+            for (RasNode x : hostnameToNodes.get(h)) {
+                hostItem.add(new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0));
+            }
+            rackResourcesSummary.addObjectResourcesItem(hostItem);
+        });
+
+        LOG.debug(
+                "Rack {}: Overall Avail [ {} ] Total [ {} ]",
+                rackId,
+                rackResourcesSummary.getAvailableResourcesOverall(),
+                rackResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            rackResourcesSummary,
+            exec,
+            (hostId) -> {
+                AtomicInteger count = scheduledCount.get(hostId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availRasNodes a list of all the nodes we want to sort
+     * @param hostId     the host-id that availNodes are a part of
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem} for nodes.
+     */
+    private Iterable<ObjectResourcesItem> sortNodes(
+            List<RasNode> availRasNodes, ExecutorDetails exec, String hostId,
+            Map<String, AtomicInteger> scheduledCount) {

Review comment:
       It feels like an overuse of AtomicInteger.  If this is only about reducing the code, is it possible to change it to `Map<String, Integer>` and use getOrDefault method, for example,
   
   ```
   perNodeScheduledCount.computeIfAbsent(superId, (sid) -> new AtomicInteger(0))
                           .getAndAdd(entry.getValue().size());
   ```
   can be changed to 
   
   ```
   perNodeScheduledCount.put(superId, perNodeScheduledCount.getOrDefault(superId, 0) + entry.getValue().size());
   ```
   And 
   ```
   (rackId) -> {
                   AtomicInteger count = scheduledCount.get(rackId);
                   if (count == null) {
                       return 0;
                   }
                   return count.get();
               }
   ```
   can be changed to 
   ```
   (rackId) -> scheduledCount.getOrDefault(rackId, 0);
   ```




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r584025739



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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;
+    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 (hostName == null) {
+                // ignore supervisors on blacklistedHosts
+                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 = computeRackToHosts(cluster, superIdToRack);
+
+        // 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;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        Set<String> retVal = new HashSet<>();
+        requestedResources.toNormalizedMap().entrySet().forEach(
+            e -> {
+                if (e.getValue() > 0.0) {
+                    retVal.add(e.getKey());
+                }
+            });
+        return retVal;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.
+     *
+     * @param resourcesSummary     contains all individual {@link ObjectResourcesItem} as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    protected Iterable<ObjectResourcesItem> sortObjectResources(
+            ObjectResourcesSummary resourcesSummary, ExecutorDetails exec, ExistingScheduleFunc existingScheduleFunc) {
+        switch (nodeSortType) {
+            case DEFAULT_RAS:
+                return sortObjectResourcesDefault(resourcesSummary, existingScheduleFunc);
+            case GENERIC_RAS:
+                return sortObjectResourcesGeneric(resourcesSummary, exec, existingScheduleFunc);
+            case COMMON:
+                return sortObjectResourcesCommon(resourcesSummary, exec, existingScheduleFunc);
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Sort objects by the following three criteria.
+     *
+     * <li>
+     *     The number executors of the topology that needs to be scheduled is already on the object (node or rack)
+     *     in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on
+     *     the same object (node or rack) as the existing executors of the topology.
+     * </li>
+     *
+     * <li>
+     *     The subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     *     resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     *     resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     *     How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     *     requested by the executor that the sorting is being done for and pulls it down if it is not.
+     *     By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     *     above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     *     resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     *     have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.
+     *     This is similar to logic used {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     * </li>
+     *
+     * <li>
+     *     The tie between two nodes with same resource availability is broken by using the node with lower minimum
+     *     percentage used. This comparison was used in {@link #sortObjectResourcesDefault(ObjectResourcesSummary, ExistingScheduleFunc)}
+     *     but here it is made subservient to modified resource availbility used in
+     *     {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     *
+     * </li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    private Iterable<ObjectResourcesItem> sortObjectResourcesCommon(
+            final ObjectResourcesSummary allResources, final ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        // Copy and modify allResources
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        final NormalizedResourceRequest requestedResources = (exec != null) ? topologyDetails.getTotalResources(exec) : null;
+        affinityBasedAllResources.getObjectResources().forEach(
+            x -> {
+                if (requestedResources != null) {
+                    // negate unrequested resources
+                    x.availableResources.updateForRareResourceAffinity(requestedResources);
+                }
+                x.minResourcePercent = availableResourcesOverall.calculateMinPercentageUsedBy(x.availableResources);
+                x.avgResourcePercent = availableResourcesOverall.calculateAveragePercentageUsedBy(x.availableResources);
+
+                LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                        x.id, x.minResourcePercent, x.avgResourcePercent,
+                        existingScheduleFunc.getNumExistingSchedule(x.id));
+            }
+        );
+
+        // Use the following comparator to sort
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double o1Avg = o1.avgResourcePercent;
+            double o2Avg = o2.avgResourcePercent;
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     * requested by the executor that the sorting is being done for and pulls it down if it is not.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     * resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     * have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesGeneric(
+            final ObjectResourcesSummary allResources, ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        affinityBasedAllResources.getObjectResources()
+                .forEach(x -> x.availableResources.updateForRareResourceAffinity(requestedResources));
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            double o1Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o1.availableResources);
+            double o2Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o2.availableResources);
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability. So we will be less likely to pick
+     * a rack that have a lot of one resource but a low amount of another.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesDefault(
+            final ObjectResourcesSummary allResources,
+            final ExistingScheduleFunc existingScheduleFunc) {
+
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        for (ObjectResourcesItem objectResources : allResources.getObjectResources()) {
+            objectResources.minResourcePercent =
+                    availableResourcesOverall.calculateMinPercentageUsedBy(objectResources.availableResources);
+            objectResources.avgResourcePercent =
+                    availableResourcesOverall.calculateAveragePercentageUsedBy(objectResources.availableResources);
+            LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                    objectResources.id, objectResources.minResourcePercent, objectResources.avgResourcePercent,
+                    existingScheduleFunc.getNumExistingSchedule(objectResources.id));
+        }
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double diff = o1.avgResourcePercent - o2.avgResourcePercent;
+            if (diff > 0.0) {
+                return -1;
+            } else if (diff < 0.0) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(allResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availHosts a collection of all the hosts we want to sort
+     * @param rackId     the rack id availNodes are a part of
+     * @return an iterable of sorted hosts.
+     */
+    private Iterable<ObjectResourcesItem> sortHosts(
+            Collection<String> availHosts, ExecutorDetails exec, String rackId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary rackResourcesSummary = new ObjectResourcesSummary("RACK");
+        availHosts.forEach(h -> {
+            ObjectResourcesItem hostItem = new ObjectResourcesItem(h);
+            for (RasNode x : hostnameToNodes.get(h)) {
+                hostItem.add(new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0));
+            }
+            rackResourcesSummary.addObjectResourcesItem(hostItem);
+        });
+
+        LOG.debug(
+                "Rack {}: Overall Avail [ {} ] Total [ {} ]",
+                rackId,
+                rackResourcesSummary.getAvailableResourcesOverall(),
+                rackResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            rackResourcesSummary,
+            exec,
+            (hostId) -> {
+                AtomicInteger count = scheduledCount.get(hostId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availRasNodes a list of all the nodes we want to sort
+     * @param hostId     the host-id that availNodes are a part of
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem} for nodes.
+     */
+    private Iterable<ObjectResourcesItem> sortNodes(
+            List<RasNode> availRasNodes, ExecutorDetails exec, String hostId,
+            Map<String, AtomicInteger> scheduledCount) {

Review comment:
       I may try int[] and see if that looks any better.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r593429125



##########
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:
       Removed




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r580351156



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceOffer.java
##########
@@ -178,6 +178,14 @@ public void updateForRareResourceAffinity(NormalizedResourceRequest requestedRes
         normalizedResources.updateForRareResourceAffinity(requestedResources.getNormalizedResources());
     }
 
+    /**
+     * If the resource is negative, then make that resource positive.
+     * This will undo effects of {@link #updateForRareResourceAffinity(NormalizedResourceRequest)}.
+     */
+    public void revertUpdatesForRareResourceAffinity() {

Review comment:
       This is not used. Can we remove?

##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/ISchedulingState.java
##########
@@ -282,6 +285,18 @@ boolean wouldFit(
      */
     Map<String, List<String>> getNetworkTopography();
 
+    /**
+     * Get host -> rack map - the inverse of networkTopography..

Review comment:
       nit: remove last `.`

##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,717 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.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, List<RasNode>> rackIdToNodes = new HashMap<>();
+    private final Map<String, List<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();
+        Map<String, String> hostToRack = cluster.getHostToRack();
+        RasNodes nodes = new RasNodes(cluster);
+        for (RasNode node: nodes.getNodes()) {
+            String superId = node.getId();
+            String hostName = node.getHostname();
+            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, id -> new ArrayList<>()).add(hostName);
+        }
+        this.greyListedSupervisorIds = cluster.getGreyListedSupervisors();
+
+        // 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);
+    }
+
+    @Override
+    public void prepare(ExecutorDetails exec) {
+        this.exec = exec;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        Set<String> retVal = new HashSet<>();
+        requestedResources.toNormalizedMap().entrySet().forEach(
+            e -> {
+                if (e.getValue() > 0.0) {
+                    retVal.add(e.getKey());
+                }
+            });
+        return retVal;
+    }
+
+    /**
+     * Scheduling uses {@link #sortAllNodes()} which eventually
+     * calls this method whose behavior can altered by setting {@link #nodeSortType}.
+     *
+     * @param resourcesSummary     contains all individual {@link ObjectResourcesItem} as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    protected Iterable<ObjectResourcesItem> sortObjectResources(
+            ObjectResourcesSummary resourcesSummary, ExecutorDetails exec, ExistingScheduleFunc existingScheduleFunc) {
+        switch (nodeSortType) {
+            case DEFAULT_RAS:
+                return sortObjectResourcesDefault(resourcesSummary, existingScheduleFunc);
+            case GENERIC_RAS:
+                return sortObjectResourcesGeneric(resourcesSummary, exec, existingScheduleFunc);
+            case COMMON:
+                return sortObjectResourcesCommon(resourcesSummary, exec, existingScheduleFunc);
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Sort objects by the following three criteria.
+     *
+     * <li>
+     *     The number executors of the topology that needs to be scheduled is already on the object (node or rack)
+     *     in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on
+     *     the same object (node or rack) as the existing executors of the topology.
+     * </li>
+     *
+     * <li>
+     *     The subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     *     resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     *     resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     *     How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     *     requested by the executor that the sorting is being done for and pulls it down if it is not.
+     *     By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     *     above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     *     resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     *     have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.
+     *     This is similar to logic used {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     * </li>
+     *
+     * <li>
+     *     The tie between two nodes with same resource availability is broken by using the node with lower minimum
+     *     percentage used. This comparison was used in {@link #sortObjectResourcesDefault(ObjectResourcesSummary, ExistingScheduleFunc)}
+     *     but here it is made subservient to modified resource availbility used in
+     *     {@link #sortObjectResourcesGeneric(ObjectResourcesSummary, ExecutorDetails, ExistingScheduleFunc)}.
+     *
+     * </li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    private Iterable<ObjectResourcesItem> sortObjectResourcesCommon(
+            final ObjectResourcesSummary allResources, final ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        // Copy and modify allResources
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        final NormalizedResourceRequest requestedResources = (exec != null) ? topologyDetails.getTotalResources(exec) : null;
+        affinityBasedAllResources.getObjectResources().forEach(
+            x -> {
+                if (requestedResources != null) {
+                    // negate unrequested resources
+                    x.availableResources.updateForRareResourceAffinity(requestedResources);
+                }
+                x.minResourcePercent = availableResourcesOverall.calculateMinPercentageUsedBy(x.availableResources);
+                x.avgResourcePercent = availableResourcesOverall.calculateAveragePercentageUsedBy(x.availableResources);
+
+                LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                        x.id, x.minResourcePercent, x.avgResourcePercent,
+                        existingScheduleFunc.getNumExistingSchedule(x.id));
+            }
+        );
+
+        // Use the following comparator to sort
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double o1Avg = o1.avgResourcePercent;
+            double o2Avg = o2.avgResourcePercent;
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * How this differs from the DefaultResourceAwareStrategy is that the percentage boosts the node or rack if it is
+     * requested by the executor that the sorting is being done for and pulls it down if it is not.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability and nodes or racks that have
+     * resources that are not requested will be ranked below . So we will be less likely to pick a rack that
+     * have a lot of one resource but a low amount of another and have a lot of resources that are not requested by the executor.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param exec                 executor for which the sorting is done
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesGeneric(
+            final ObjectResourcesSummary allResources, ExecutorDetails exec,
+            final ExistingScheduleFunc existingScheduleFunc) {
+        ObjectResourcesSummary affinityBasedAllResources = new ObjectResourcesSummary(allResources);
+        NormalizedResourceRequest requestedResources = topologyDetails.getTotalResources(exec);
+        affinityBasedAllResources.getObjectResources()
+                .forEach(x -> x.availableResources.updateForRareResourceAffinity(requestedResources));
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            double o1Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o1.availableResources);
+            double o2Avg = availableResourcesOverall.calculateAveragePercentageUsedBy(o2.availableResources);
+            if (o1Avg > o2Avg) {
+                return -1;
+            } else if (o1Avg < o2Avg) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(affinityBasedAllResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Sort objects by the following two criteria.
+     *
+     * <li>the number executors of the topology that needs to be scheduled is already on the
+     * object (node or rack) in descending order. The reasoning to sort based on criterion 1 is so we schedule the rest
+     * of a topology on the same object (node or rack) as the existing executors of the topology.</li>
+     *
+     * <li>the subordinate/subservient resource availability percentage of a rack in descending order We calculate the
+     * resource availability percentage by dividing the resource availability of the object (node or rack) by the
+     * resource availability of the entire rack or cluster depending on if object references a node or a rack.
+     * By doing this calculation, objects (node or rack) that have exhausted or little of one of the resources mentioned
+     * above will be ranked after racks that have more balanced resource availability. So we will be less likely to pick
+     * a rack that have a lot of one resource but a low amount of another.</li>
+     *
+     * @param allResources         contains all individual ObjectResources as well as cumulative stats
+     * @param existingScheduleFunc a function to get existing executors already scheduled on this object
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem}
+     */
+    @Deprecated
+    private Iterable<ObjectResourcesItem> sortObjectResourcesDefault(
+            final ObjectResourcesSummary allResources,
+            final ExistingScheduleFunc existingScheduleFunc) {
+
+        final NormalizedResourceOffer availableResourcesOverall = allResources.getAvailableResourcesOverall();
+        for (ObjectResourcesItem objectResources : allResources.getObjectResources()) {
+            objectResources.minResourcePercent =
+                    availableResourcesOverall.calculateMinPercentageUsedBy(objectResources.availableResources);
+            objectResources.avgResourcePercent =
+                    availableResourcesOverall.calculateAveragePercentageUsedBy(objectResources.availableResources);
+            LOG.trace("for {}: minResourcePercent={}, avgResourcePercent={}, numExistingSchedule={}",
+                    objectResources.id, objectResources.minResourcePercent, objectResources.avgResourcePercent,
+                    existingScheduleFunc.getNumExistingSchedule(objectResources.id));
+        }
+
+        Comparator<ObjectResourcesItem> comparator = (o1, o2) -> {
+            int execsScheduled1 = existingScheduleFunc.getNumExistingSchedule(o1.id);
+            int execsScheduled2 = existingScheduleFunc.getNumExistingSchedule(o2.id);
+            if (execsScheduled1 > execsScheduled2) {
+                return -1;
+            } else if (execsScheduled1 < execsScheduled2) {
+                return 1;
+            }
+            if (o1.minResourcePercent > o2.minResourcePercent) {
+                return -1;
+            } else if (o1.minResourcePercent < o2.minResourcePercent) {
+                return 1;
+            }
+            double diff = o1.avgResourcePercent - o2.avgResourcePercent;
+            if (diff > 0.0) {
+                return -1;
+            } else if (diff < 0.0) {
+                return 1;
+            }
+            return o1.id.compareTo(o2.id);
+        };
+        TreeSet<ObjectResourcesItem> sortedObjectResources = new TreeSet<>(comparator);
+        sortedObjectResources.addAll(allResources.getObjectResources());
+        LOG.debug("Sorted Object Resources: {}", sortedObjectResources);
+        return sortedObjectResources;
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availHosts a list of all the hosts we want to sort
+     * @param rackId     the rack id availNodes are a part of
+     * @return an iterable of sorted hosts.
+     */
+    private Iterable<ObjectResourcesItem> sortHosts(
+            List<String> availHosts, ExecutorDetails exec, String rackId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary rackResourcesSummary = new ObjectResourcesSummary("RACK");
+        availHosts.forEach(h -> {
+            ObjectResourcesItem hostItem = new ObjectResourcesItem(h);
+            for (RasNode x : hostnameToNodes.get(h)) {
+                hostItem.add(new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0));
+            }
+            rackResourcesSummary.addObjectResourcesItem(hostItem);
+        });
+
+        LOG.debug(
+                "Rack {}: Overall Avail [ {} ] Total [ {} ]",
+                rackId,
+                rackResourcesSummary.getAvailableResourcesOverall(),
+                rackResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            rackResourcesSummary,
+            exec,
+            (hostId) -> {
+                AtomicInteger count = scheduledCount.get(hostId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    /**
+     * Nodes are sorted by two criteria.
+     *
+     * <p>1) the number executors of the topology that needs to be scheduled is already on the node in
+     * descending order. The reasoning to sort based on criterion 1 is so we schedule the rest of a topology on the same node as the
+     * existing executors of the topology.
+     *
+     * <p>2) the subordinate/subservient resource availability percentage of a node in descending
+     * order We calculate the resource availability percentage by dividing the resource availability that have exhausted or little of one of
+     * the resources mentioned above will be ranked after on the node by the resource availability of the entire rack By doing this
+     * calculation, nodes nodes that have more balanced resource availability. So we will be less likely to pick a node that have a lot of
+     * one resource but a low amount of another.
+     *
+     * @param availRasNodes a list of all the nodes we want to sort
+     * @param hostId     the host-id that availNodes are a part of
+     * @return an {@link Iterable} of sorted {@link ObjectResourcesItem} for nodes.
+     */
+    private Iterable<ObjectResourcesItem> sortNodes(
+            List<RasNode> availRasNodes, ExecutorDetails exec, String hostId,
+            Map<String, AtomicInteger> scheduledCount) {
+        ObjectResourcesSummary hostResourcesSummary = new ObjectResourcesSummary("HOST");
+        availRasNodes.forEach(x ->
+                hostResourcesSummary.addObjectResourcesItem(
+                        new ObjectResourcesItem(x.getId(), x.getTotalAvailableResources(), x.getTotalResources(), 0, 0)
+                )
+        );
+
+        LOG.debug(
+            "Host {}: Overall Avail [ {} ] Total [ {} ]",
+            hostId,
+            hostResourcesSummary.getAvailableResourcesOverall(),
+            hostResourcesSummary.getTotalResourcesOverall());
+
+        return sortObjectResources(
+            hostResourcesSummary,
+            exec,
+            (superId) -> {
+                AtomicInteger count = scheduledCount.get(superId);
+                if (count == null) {
+                    return 0;
+                }
+                return count.get();
+            });
+    }
+
+    protected List<String> makeHostToNodeIds(List<String> hosts) {
+        if (hosts == null) {
+            return Collections.emptyList();
+        }
+        List<String> ret = new ArrayList<>(hosts.size());
+        for (String host: hosts) {
+            List<RasNode> nodes = hostnameToNodes.get(host);
+            if (nodes != null) {
+                for (RasNode node : nodes) {
+                    ret.add(node.getId());
+                }
+            }
+        }
+        return ret;
+    }
+
+    private class LazyNodeSortingIterator implements Iterator<String> {
+        private final LazyNodeSorting parent;
+        private final Iterator<ObjectResourcesItem> rackIterator;
+        private Iterator<ObjectResourcesItem> hostIterator;
+        private Iterator<ObjectResourcesItem> nodeIterator;
+        private String nextValueFromNode = null;
+        private final Iterator<String> pre;
+        private final Iterator<String> post;
+        private final Set<String> skip;
+
+        LazyNodeSortingIterator(LazyNodeSorting parent, Iterable<ObjectResourcesItem> sortedRacks) {
+            this.parent = parent;
+            rackIterator = sortedRacks.iterator();
+            pre = favoredNodeIds.iterator();
+            post = Stream.concat(unFavoredNodeIds.stream(), greyListedSupervisorIds.stream())
+                            .collect(Collectors.toList())
+                            .iterator();
+            skip = parent.skippedNodeIds;
+        }
+
+        private Iterator<ObjectResourcesItem> getNodeIterator() {
+            if (nodeIterator != null && nodeIterator.hasNext()) {
+                return nodeIterator;
+            }
+            //need to get the next host/node iterator
+            if (hostIterator != null && hostIterator.hasNext()) {
+                ObjectResourcesItem host = hostIterator.next();
+                final String hostId = host.id;
+                nodeIterator = parent.getSortedNodesForHost(hostId).iterator();
+                return nodeIterator;
+            }
+            if (rackIterator.hasNext()) {
+                ObjectResourcesItem rack = rackIterator.next();
+                final String rackId = rack.id;
+                hostIterator = parent.getSortedHostsForRack(rackId).iterator();
+                ObjectResourcesItem host = hostIterator.next();
+                final String hostId = host.id;
+                nodeIterator = parent.getSortedNodesForHost(hostId).iterator();
+                return nodeIterator;
+            }
+
+            return null;
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (pre.hasNext()) {
+                return true;
+            }
+            if (nextValueFromNode != null) {
+                return true;
+            }
+            while (true) {
+                //For the node we don't know if we have another one unless we look at the contents
+                Iterator<ObjectResourcesItem> nodeIterator = getNodeIterator();
+                if (nodeIterator == null || !nodeIterator.hasNext()) {
+                    break;
+                }
+                String tmp = nodeIterator.next().id;
+                if (!skip.contains(tmp)) {
+                    nextValueFromNode = tmp;
+                    return true;
+                }
+            }
+            return post.hasNext();
+        }
+
+        @Override
+        public String next() {
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            if (pre.hasNext()) {
+                return pre.next();
+            }
+            if (nextValueFromNode != null) {
+                String tmp = nextValueFromNode;
+                nextValueFromNode = null;
+                return tmp;
+            }
+            return post.next();
+        }
+    }
+
+    private class LazyNodeSorting implements Iterable<String> {
+        private final Map<String, AtomicInteger> perHostScheduledCount = new HashMap<>();
+        private final Map<String, AtomicInteger> perNodeScheduledCount = new HashMap<>();
+        private final Iterable<ObjectResourcesItem> sortedRacks;
+        private final Map<String, Iterable<ObjectResourcesItem>> cachedHosts = new HashMap<>();
+        private final Map<String, Iterable<ObjectResourcesItem>> cachedNodesByHost = new HashMap<>();
+        private final ExecutorDetails exec;
+        private final Set<String> skippedNodeIds = new HashSet<>();
+
+        LazyNodeSorting(ExecutorDetails exec) {
+            this.exec = exec;
+            skippedNodeIds.addAll(favoredNodeIds);
+            skippedNodeIds.addAll(unFavoredNodeIds);
+            skippedNodeIds.addAll(greyListedSupervisorIds);
+
+            String topoId = topologyDetails.getId();
+            SchedulerAssignment assignment = cluster.getAssignmentById(topoId);
+            if (assignment != null) {
+                for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> entry :
+                    assignment.getSlotToExecutors().entrySet()) {
+                    String superId = entry.getKey().getNodeId();
+                    String hostId = nodeIdToHostname.get(superId);
+                    perHostScheduledCount.computeIfAbsent(hostId, id -> new AtomicInteger(0))
+                        .getAndAdd(entry.getValue().size());
+                    perNodeScheduledCount.computeIfAbsent(superId, id -> new AtomicInteger(0))
+                        .getAndAdd(entry.getValue().size());
+                }
+            }
+            sortedRacks = getSortedRacks();
+        }
+
+        private Iterable<ObjectResourcesItem> getSortedHostsForRack(String rackId) {
+            return cachedHosts.computeIfAbsent(rackId,
+                id -> sortHosts(rackIdToHosts.getOrDefault(id, Collections.emptyList()), exec, id, perHostScheduledCount));
+        }
+
+        private Iterable<ObjectResourcesItem> getSortedNodesForHost(String hostId) {
+            return cachedNodesByHost.computeIfAbsent(hostId,
+                id -> sortNodes(hostnameToNodes.getOrDefault(id, Collections.emptyList()), exec, id, perNodeScheduledCount));
+        }
+
+        @Override
+        public Iterator<String> iterator() {
+            return new LazyNodeSortingIterator(this, sortedRacks);
+        }
+    }
+
+    @Override
+    public Iterable<String> sortAllNodes() {
+        //LOG.info("sortAllNodes():cachedLazyNodeIterators.size={}, execRequestResourceKey={}",

Review comment:
       Delete if not needed?




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r584024741



##########
File path: storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestDefaultResourceAwareStrategy.java
##########
@@ -905,9 +911,9 @@ public void testMultipleRacksWithFavoritism() {
 
         rs.prepareForScheduling(cluster, topo1);
         INodeSorter nodeSorter = new NodeSorter(cluster, topo1, BaseResourceAwareStrategy.NodeSortType.DEFAULT_RAS);
-        TreeSet<ObjectResourcesItem> sortedRacks= nodeSorter.sortRacks(null);
+        nodeSorter.prepare(null);
+        Iterable<ObjectResourcesItem> sortedRacks= nodeSorter.getSortedRacks();
 
-        Assert.assertEquals("# of racks sorted", 5, sortedRacks.size());

Review comment:
       The returned object it iterable. The check happens implicitly when each of the five elements are checked following.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r581621856



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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 {

Review comment:
       I want to remove NodeSorter when the deprecated sort type is removed. Not very extension friendly classes.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r593424759



##########
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:
       fixed




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r581617481



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResources.java
##########
@@ -384,14 +391,29 @@ public double calculateMinPercentageUsedBy(NormalizedResources used, double tota
 
     /**
      * If a node or rack has a kind of resource not in a request, make that resource negative so when sorting that node or rack will
-     * be less likely to be selected.
+     * be less likely to be selected. If the resource is in the request, make that resource positive.
      * @param request the requested resources.
      */
     public void updateForRareResourceAffinity(NormalizedResources request) {
         int length = Math.min(this.otherResources.length, request.otherResources.length);
         for (int i = 0; i < length; i++) {
-            if (request.getResourceAt(i) == 0.0) {
-                this.otherResources[i] = -1 * this.otherResources[i];
+            if (request.getResourceAt(i) == 0.0 && this.otherResources[i] > 0.0) {
+                this.otherResources[i] = -this.otherResources[i]; // make negative
+            } else if (request.getResourceAt(i) > 0.0 && this.otherResources[i] < 0.0) {

Review comment:
       The goal in this code is to make unrequested resource negative (but also requested resource positive). So when processing a new executor with different resource requirements, it can be reapplied and re-sort resulting elements. With earlier code, this was a "use-once-only" method on the "otherResource". 
   
   With the change above, the object can be reused.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582929248



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/ISchedulingState.java
##########
@@ -339,4 +354,31 @@ boolean wouldFit(
      * Get the nimbus configuration.
      */
     Map<String, Object> getConf();
+
+    /**
+     * Determine the list of racks on which topologyIds have been assigned. Note that the returned set
+     * may contain {@link DNSToSwitchMapping#DEFAULT_RACK} if {@link #getHostToRack()} is null or
+     * does not contain the assigned host.
+     *
+     * @param topologyIds for which assignments are examined.
+     * @return set of racks on which assignments have been made.
+     */
+    default Set<String> getAssignedRacks(String... topologyIds) {

Review comment:
       It makes the interface unnecessary bigger since this method is only used in test cases. I understand devs don't need to implement this method. But they need to maintain the code. My personal opinion is that we shouldn't add code that is not necessary. With that being said, I won't block you on this if you insist.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r583807959



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceOffer.java
##########
@@ -178,6 +178,14 @@ public void updateForRareResourceAffinity(NormalizedResourceRequest requestedRes
         normalizedResources.updateForRareResourceAffinity(requestedResources.getNormalizedResources());
     }
 
+    /**
+     * If the resource is negative, then make that resource positive.
+     * This will undo effects of {@link #updateForRareResourceAffinity(NormalizedResourceRequest)}.
+     */
+    public void revertUpdatesForRareResourceAffinity() {

Review comment:
       Yes. I can add it when I need it later.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r581620708



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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;
+    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 (hostName == null) {
+                // ignore supervisors on blacklistedHosts
+                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 = computeRackToHosts(cluster, superIdToRack);
+
+        // 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;
+    }
+
+    /**
+     * Get a key corresponding to this executor resource request. The key contains all non-zero resources requested by
+     * the executor.
+     *
+     * <p>This key can be used to retrieve pre-sorted list of racks/hosts/nodes. So executors with similar set of
+     * rare resource request will be reuse the same cached list instead of creating a new sorted list of nodes.</p>
+     *
+     * @param exec executor whose requested resources are being examined.
+     * @return a set of resource names that have values greater that zero.
+     */
+    public Set<String> getExecRequestKey(@Nonnull ExecutorDetails exec) {

Review comment:
       This was going to be used for sort cacheing, but looks like sort is fast enough as it is. I will remove it.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
Ethanlm commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r582098987



##########
File path: storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/sorter/NodeSorterHostProximity.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 javax.annotation.Nonnull;
+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.SupervisorDetails;
+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 {

Review comment:
       This is related to one of the comment above. If the sort type is no longer used and if we want to remove the backwards compatibility like GenericResourceAwareStrategyOld, we can remove all these related code (doesn't have to be in this PR)




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
bipinprasad commented on a change in pull request #3379:
URL: https://github.com/apache/storm/pull/3379#discussion_r593425870



##########
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:
       fixed




----------------------------------------------------------------
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