You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by da...@apache.org on 2015/12/21 15:46:29 UTC

[01/23] storm git commit: [STORM-898] - Add priorities and per user resource guarantees to Resource Aware Scheduler

Repository: storm
Updated Branches:
  refs/heads/master 8f9ed06db -> c7164ccd4


[STORM-898] - Add priorities and per user resource guarantees to Resource Aware Scheduler


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

Branch: refs/heads/master
Commit: 0d34abf9ca19f381578053497b103f2bb8a6cff2
Parents: 6babbb0
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Fri Dec 4 13:06:26 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Fri Dec 4 13:06:26 2015 -0600

----------------------------------------------------------------------
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   6 +-
 .../jvm/backtype/storm/scheduler/Cluster.java   |  22 ++
 .../backtype/storm/scheduler/Topologies.java    |   6 +-
 .../storm/scheduler/TopologyDetails.java        |  36 ++-
 .../resource/ResourceAwareScheduler.java        | 263 ++++++++++++-------
 .../storm/scheduler/resource/ResourceUtils.java |  51 ++++
 .../backtype/storm/scheduler/resource/User.java | 239 +++++++++++++++++
 .../storm/validation/ConfigValidation.java      |  34 ++-
 8 files changed, 551 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/0d34abf9/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index 9834f25..e22161b 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -540,12 +540,16 @@
         topology (read-storm-topology-as-nimbus storm-id blob-store)
         executor->component (->> (compute-executor->component nimbus storm-id)
                                  (map-key (fn [[start-task end-task]]
-                                            (ExecutorDetails. (int start-task) (int end-task)))))]
+                                            (ExecutorDetails. (int start-task) (int end-task)))))
+        launch-time-secs (if storm-base (:launch-time-secs storm-base)
+                           (throw
+                             (NotAliveException. (str storm-id))))]
     (TopologyDetails. storm-id
                       topology-conf
                       topology
                       (:num-workers storm-base)
                       executor->component
+                      launch-time-secs
                       )))
 
 ;; Does not assume that clocks are synchronized. Executor heartbeat is only used so that

http://git-wip-us.apache.org/repos/asf/storm/blob/0d34abf9/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
index ff2b233..f4d12d8 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
@@ -436,6 +436,28 @@ public class Cluster {
         return this.supervisors;
     }
 
+    /**
+     * Get the total amount of CPU resources in cluster
+     */
+    public double getClusterTotalCPUResource() {
+        double sum = 0.0;
+        for(SupervisorDetails sup: this.supervisors.values()) {
+            sum += sup.getTotalCPU();
+        }
+        return sum;
+    }
+
+    /**
+     * Get the total amount of memory resources in cluster
+     */
+    public double getClusterTotalMemoryResource() {
+        double sum = 0.0;
+        for(SupervisorDetails sup: this.supervisors.values()) {
+            sum += sup.getTotalMemory();
+        }
+        return sum;
+    }
+
     /*
     * Note: Make sure the proper conf was passed into the Cluster constructor before calling this function
     * It tries to load the proper network topography detection plugin specified in the config.

http://git-wip-us.apache.org/repos/asf/storm/blob/0d34abf9/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
index 0828a73..443bf3f 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
@@ -29,10 +29,10 @@ public class Topologies {
     Map<String, Map<String, Component>> _allComponents;
 
     public Topologies(Map<String, TopologyDetails> topologies) {
-        if(topologies==null) topologies = new HashMap<>();
-        this.topologies = new HashMap<>(topologies.size());
+        if(topologies==null) topologies = new HashMap();
+        this.topologies = new HashMap<String, TopologyDetails>(topologies.size());
         this.topologies.putAll(topologies);
-        this.nameToId = new HashMap<>(topologies.size());
+        this.nameToId = new HashMap<String, String>(topologies.size());
         
         for (Map.Entry<String, TopologyDetails> entry : topologies.entrySet()) {
             TopologyDetails topology = entry.getValue();

http://git-wip-us.apache.org/repos/asf/storm/blob/0d34abf9/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
index 95aa5c8..992a522 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
@@ -48,6 +48,10 @@ public class TopologyDetails {
     //Max heap size for a worker used by topology
     private Double topologyWorkerMaxHeapSize;
 
+    private Integer topologyPriority;
+
+    private int launchTime;
+
     private static final Logger LOG = LoggerFactory.getLogger(TopologyDetails.class);
 
     public TopologyDetails(String topologyId, Map topologyConf, StormTopology topology, int numWorkers) {
@@ -57,12 +61,14 @@ public class TopologyDetails {
         this.numWorkers = numWorkers;
     }
 
-    public TopologyDetails(String topologyId, Map topologyConf, StormTopology topology, int numWorkers, Map<ExecutorDetails, String> executorToComponents) {
+    public TopologyDetails(String topologyId, Map topologyConf, StormTopology topology,
+                           int numWorkers, Map<ExecutorDetails, String> executorToComponents, int launchTime) {
         this(topologyId, topologyConf, topology, numWorkers);
         this.executorToComponent = new HashMap<>(0);
         if (executorToComponents != null) {
             this.executorToComponent.putAll(executorToComponents);
         }
+        this.launchTime = launchTime;
         this.initResourceList();
         this.initConfigs();
     }
@@ -424,6 +430,7 @@ public class TopologyDetails {
      */
     private void initConfigs() {
         this.topologyWorkerMaxHeapSize = Utils.getDouble(this.topologyConf.get(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB), null);
+        this.topologyPriority = Utils.getInt(this.topologyConf.get(Config.TOPOLOGY_PRIORITY), null);
     }
 
     /**
@@ -433,4 +440,31 @@ public class TopologyDetails {
     public Double getTopologyWorkerMaxHeapSize() {
         return this.topologyWorkerMaxHeapSize;
     }
+
+    public String getTopologySubmitter() {
+       return (String)this.topologyConf.get(Config.TOPOLOGY_SUBMITTER_USER);
+    }
+
+    public int getTopologyPriority() {
+       return this.topologyPriority;
+    }
+    public int getLaunchTime() {
+        return this.launchTime;
+    }
+
+    public int getUpTime() {
+        return Time.currentTimeSecs() - this.launchTime;
+    }
+
+    @Override
+    public String toString() {
+        return "Name: " + this.getName() + " Priority: " + this.getTopologyPriority()
+                + " Uptime: " + this.getUpTime() + " CPU: " + this.getTotalRequestedCpu()
+                + " Memory: " + (this.getTotalRequestedMemOffHeap() + this.getTotalRequestedMemOnHeap());
+    }
+
+    @Override
+    public int hashCode() {
+        return this.topologyId.hashCode();
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/0d34abf9/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
index c7c7867..65d1841 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
@@ -19,6 +19,7 @@
 package backtype.storm.scheduler.resource;
 
 import backtype.storm.Config;
+import backtype.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -33,82 +34,111 @@ import backtype.storm.scheduler.resource.strategies.ResourceAwareStrategy;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.LinkedList;
 import java.util.Map;
 import java.util.Set;
 
 public class ResourceAwareScheduler implements IScheduler {
+
+    private Map<String, User> userMap = new HashMap<String, User>();
+    private Cluster cluster;
+    private Topologies topologies;
+
+
+    @SuppressWarnings("rawtypes")
+    private Map conf;
+
     private static final Logger LOG = LoggerFactory
             .getLogger(ResourceAwareScheduler.class);
-    @SuppressWarnings("rawtypes")
-    private Map _conf;
 
     @Override
     public void prepare(Map conf) {
-        _conf = conf;
+        this.conf = conf;
     }
 
     @Override
     public void schedule(Topologies topologies, Cluster cluster) {
-        LOG.debug("\n\n\nRerunning ResourceAwareScheduler...");
-
-        ResourceAwareStrategy RAStrategy = new ResourceAwareStrategy(cluster, topologies);
-        LOG.debug(printScheduling(cluster, topologies));
-
-        for (TopologyDetails td : topologies.getTopologies()) {
-            String topId = td.getId();
-            Map<WorkerSlot, Collection<ExecutorDetails>> schedulerAssignmentMap;
-            if (cluster.getUnassignedExecutors(td).size() > 0) {
-                LOG.debug("/********Scheduling topology {} ************/", topId);
-
-                schedulerAssignmentMap = RAStrategy.schedule(td);
-
-                double requestedMemOnHeap = td.getTotalRequestedMemOnHeap();
-                double requestedMemOffHeap = td.getTotalRequestedMemOffHeap();
-                double requestedCpu = td.getTotalRequestedCpu();
-                double assignedMemOnHeap = 0.0;
-                double assignedMemOffHeap = 0.0;
-                double assignedCpu = 0.0;
-
-                if (schedulerAssignmentMap != null) {
-                    try {
-                        Set<String> nodesUsed = new HashSet<String>();
-                        int assignedWorkers = schedulerAssignmentMap.keySet().size();
-                        for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> workerToTasksEntry : schedulerAssignmentMap.entrySet()) {
-                            WorkerSlot targetSlot = workerToTasksEntry.getKey();
-                            Collection<ExecutorDetails> execsNeedScheduling = workerToTasksEntry.getValue();
-                            RAS_Node targetNode = RAStrategy.idToNode(targetSlot.getNodeId());
-                            targetNode.assign(targetSlot, td, execsNeedScheduling, cluster);
-                            LOG.debug("ASSIGNMENT    TOPOLOGY: {}  TASKS: {} To Node: {} on Slot: {}",
-                                    td.getName(), execsNeedScheduling, targetNode.getHostname(), targetSlot.getPort());
-                            if (!nodesUsed.contains(targetNode.getId())) {
-                                nodesUsed.add(targetNode.getId());
-                            }
-                            assignedMemOnHeap += targetSlot.getAllocatedMemOnHeap();
-                            assignedMemOffHeap += targetSlot.getAllocatedMemOffHeap();
-                            assignedCpu += targetSlot.getAllocatedCpu();
+        LOG.info("\n\n\nRerunning ResourceAwareScheduler...");
+        LOG.debug(ResourceUtils.printScheduling(cluster, topologies));
+
+        this.initialize(topologies, cluster);
+
+
+        LOG.info("UserMap:\n{}", this.userMap);
+        for(User user : this.getUserMap().values()) {
+            LOG.info(user.getDetailedInfo());
+        }
+
+        LOG.info("getNextUser: {}", this.getNextUser());
+
+        while(true) {
+            User nextUser = this.getNextUser();
+            if(nextUser == null){
+                break;
+            }
+            TopologyDetails td = nextUser.getNextTopologyToSchedule();
+            scheduleTopology(td);
+        }
+    }
+
+    private void scheduleTopology(TopologyDetails td) {
+        ResourceAwareStrategy RAStrategy = new ResourceAwareStrategy(this.cluster, this.topologies);
+        if (cluster.needsScheduling(td) && cluster.getUnassignedExecutors(td).size() > 0) {
+            LOG.info("/********Scheduling topology {} from User {}************/", td.getName(), td.getTopologySubmitter());
+            LOG.info("{}", this.userMap.get(td.getTopologySubmitter()).getDetailedInfo());
+            LOG.info("{}", User.getResourcePoolAverageUtilizationForUsers(this.userMap.values()));
+
+            Map<WorkerSlot, Collection<ExecutorDetails>> schedulerAssignmentMap = RAStrategy.schedule(td);
+
+            double requestedMemOnHeap = td.getTotalRequestedMemOnHeap();
+            double requestedMemOffHeap = td.getTotalRequestedMemOffHeap();
+            double requestedCpu = td.getTotalRequestedCpu();
+            double assignedMemOnHeap = 0.0;
+            double assignedMemOffHeap = 0.0;
+            double assignedCpu = 0.0;
+
+            if (schedulerAssignmentMap != null) {
+                try {
+                    Set<String> nodesUsed = new HashSet<String>();
+                    int assignedWorkers = schedulerAssignmentMap.keySet().size();
+                    for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> workerToTasksEntry : schedulerAssignmentMap.entrySet()) {
+                        WorkerSlot targetSlot = workerToTasksEntry.getKey();
+                        Collection<ExecutorDetails> execsNeedScheduling = workerToTasksEntry.getValue();
+                        RAS_Node targetNode = RAStrategy.idToNode(targetSlot.getNodeId());
+                        targetNode.assign(targetSlot, td, execsNeedScheduling, this.cluster);
+                        LOG.debug("ASSIGNMENT    TOPOLOGY: {}  TASKS: {} To Node: {} on Slot: {}",
+                                td.getName(), execsNeedScheduling, targetNode.getHostname(), targetSlot.getPort());
+                        if (!nodesUsed.contains(targetNode.getId())) {
+                            nodesUsed.add(targetNode.getId());
                         }
-                        LOG.debug("Topology: {} assigned to {} nodes on {} workers", td.getId(), nodesUsed.size(), assignedWorkers);
-                        cluster.setStatus(td.getId(), "Fully Scheduled");
-                    } catch (IllegalStateException ex) {
-                        LOG.error(ex.toString());
-                        LOG.error("Unsuccessful in scheduling", td.getId());
-                        cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
+                        assignedMemOnHeap += targetSlot.getAllocatedMemOnHeap();
+                        assignedMemOffHeap += targetSlot.getAllocatedMemOffHeap();
+                        assignedCpu += targetSlot.getAllocatedCpu();
                     }
-                } else {
+                    LOG.debug("Topology: {} assigned to {} nodes on {} workers", td.getId(), nodesUsed.size(), assignedWorkers);
+                    this.cluster.setStatus(td.getId(), "Fully Scheduled");
+                    this.getUser(td.getTopologySubmitter()).moveTopoFromPendingToRunning(td);
+                    LOG.info("getNextUser: {}", this.getNextUser());
+                } catch (IllegalStateException ex) {
+                    LOG.error(ex.toString());
                     LOG.error("Unsuccessful in scheduling", td.getId());
-                    cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
+                    this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
+                    this.getUser(td.getTopologySubmitter()).moveTopoFromPendingToAttempted(td);
                 }
-                Double[] resources = {requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
-                        assignedMemOnHeap, assignedMemOffHeap, assignedCpu};
-                LOG.debug("setResources for {}: requested on-heap mem, off-heap mem, cpu: {} {} {} " +
-                        "assigned on-heap mem, off-heap mem, cpu: {} {} {}",
-                        td.getId(), requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
-                        assignedMemOnHeap, assignedMemOffHeap, assignedCpu);
-                cluster.setResources(td.getId(), resources);
             } else {
-                cluster.setStatus(td.getId(), "Fully Scheduled");
+                LOG.error("Unsuccessful in scheduling {}", td.getId());
+                this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
+                this.getUser(td.getTopologySubmitter()).moveTopoFromPendingToAttempted(td);
             }
+            Double[] resources = {requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
+                    assignedMemOnHeap, assignedMemOffHeap, assignedCpu};
+            LOG.debug("setResources for {}: requested on-heap mem, off-heap mem, cpu: {} {} {} " +
+                            "assigned on-heap mem, off-heap mem, cpu: {} {} {}",
+                    td.getId(), requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
+                    assignedMemOnHeap, assignedMemOffHeap, assignedCpu);
+            this.cluster.setResources(td.getId(), resources);
+        } else {
+            LOG.warn("Topology {} already scheduled!", td.getName());
+            this.cluster.setStatus(td.getId(), "Fully Scheduled");
         }
         updateSupervisorsResources(cluster, topologies);
     }
@@ -127,57 +157,102 @@ public class ResourceAwareScheduler implements IScheduler {
         }
         cluster.setSupervisorsResourcesMap(supervisors_resources);
     }
+    public User getUser(String user) {
+        return this.userMap.get(user);
+    }
+
+    public Map<String, User> getUserMap() {
+        return this.userMap;
+    }
 
-    private Map<String, Double> getUserConf() {
-        Map<String, Double> ret = new HashMap<String, Double>();
-        ret.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB,
-                (Double) _conf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB));
-        ret.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB,
-                (Double) _conf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB));
+    public User getNextUser() {
+        Double least = Double.MAX_VALUE;
+        User ret = null;
+        for(User user : this.userMap.values()) {
+            if(user.hasTopologyNeedSchedule()) {
+                Double userResourcePoolAverageUtilization = user.getResourcePoolAverageUtilization();
+                if (least > userResourcePoolAverageUtilization) {
+                    ret = user;
+                    least = userResourcePoolAverageUtilization;
+                } else if (least == userResourcePoolAverageUtilization) {
+                    double currentCpuPercentage = ret.getCPUResourceGuaranteed()/this.cluster.getClusterTotalCPUResource();
+                    double currentMemoryPercentage = ret.getMemoryResourceGuaranteed()/this.cluster.getClusterTotalMemoryResource();
+                    double currentAvgPercentage = (currentCpuPercentage + currentMemoryPercentage) / 2.0;
+
+                    double userCpuPercentage = user.getCPUResourceGuaranteed()/this.cluster.getClusterTotalCPUResource();
+                    double userMemoryPercentage = user.getMemoryResourceGuaranteed()/this.cluster.getClusterTotalMemoryResource();
+                    double userAvgPercentage = (userCpuPercentage + userMemoryPercentage) / 2.0;
+                    if(userAvgPercentage > currentAvgPercentage) {
+                        ret = user;
+                        least = userResourcePoolAverageUtilization;
+                    }
+                }
+            }
+        }
         return ret;
     }
 
     /**
-     * print scheduling for debug purposes
-     * @param cluster
+     * Intialize scheduling and running queues
      * @param topologies
+     * @param cluster
      */
-    public String printScheduling(Cluster cluster, Topologies topologies) {
-        StringBuilder str = new StringBuilder();
-        Map<String, Map<String, Map<WorkerSlot, Collection<ExecutorDetails>>>> schedulingMap = new HashMap<String, Map<String, Map<WorkerSlot, Collection<ExecutorDetails>>>>();
+    private void initUsers(Topologies topologies, Cluster cluster) {
+
+        Map<String, Map<String, Double>> userResourcePools = this.getUserResourcePools();
+        LOG.info("userResourcePools: {}", userResourcePools);
+
         for (TopologyDetails topo : topologies.getTopologies()) {
-            if (cluster.getAssignmentById(topo.getId()) != null) {
-                for (Map.Entry<ExecutorDetails, WorkerSlot> entry : cluster.getAssignmentById(topo.getId()).getExecutorToSlot().entrySet()) {
-                    WorkerSlot slot = entry.getValue();
-                    String nodeId = slot.getNodeId();
-                    ExecutorDetails exec = entry.getKey();
-                    if (schedulingMap.containsKey(nodeId) == false) {
-                        schedulingMap.put(nodeId, new HashMap<String, Map<WorkerSlot, Collection<ExecutorDetails>>>());
-                    }
-                    if (schedulingMap.get(nodeId).containsKey(topo.getId()) == false) {
-                        schedulingMap.get(nodeId).put(topo.getId(), new HashMap<WorkerSlot, Collection<ExecutorDetails>>());
-                    }
-                    if (schedulingMap.get(nodeId).get(topo.getId()).containsKey(slot) == false) {
-                        schedulingMap.get(nodeId).get(topo.getId()).put(slot, new LinkedList<ExecutorDetails>());
-                    }
-                    schedulingMap.get(nodeId).get(topo.getId()).get(slot).add(exec);
-                }
+            String topologySubmitter = topo.getTopologySubmitter();
+            if(!this.userMap.containsKey(topologySubmitter)) {
+                this.userMap.put(topologySubmitter, new User(topologySubmitter, userResourcePools.get(topologySubmitter)));
+            }
+            if(cluster.getUnassignedExecutors(topo).size() >= topo.getExecutors().size()) {
+                this.userMap.get(topologySubmitter).addTopologyToPendingQueue(topo);
+            } else {
+                this.userMap.get(topologySubmitter).addTopologyToRunningQueue(topo);
             }
         }
+    }
 
-        for (Map.Entry<String, Map<String, Map<WorkerSlot, Collection<ExecutorDetails>>>> entry : schedulingMap.entrySet()) {
-            if (cluster.getSupervisorById(entry.getKey()) != null) {
-                str.append("/** Node: " + cluster.getSupervisorById(entry.getKey()).getHost() + "-" + entry.getKey() + " **/\n");
-            } else {
-                str.append("/** Node: Unknown may be dead -" + entry.getKey() + " **/\n");
+    private void initialize(Topologies topologies, Cluster cluster) {
+        initUsers(topologies, cluster);
+        this.cluster = cluster;
+        this.topologies = topologies;
+    }
+
+    /**
+     * Get resource guarantee configs
+     * @return
+     */
+    private Map<String, Map<String, Double>> getUserResourcePools() {
+
+        Map<String, Map<String, Number>> raw =  (Map<String, Map<String, Number>>)this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS);
+        Map<String, Map<String, Double>> ret =  (Map<String, Map<String, Double>>)this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS);
+
+        if (raw == null) {
+            ret = new  HashMap<String, Map<String, Double>>();
+        } else {
+            for(Map.Entry<String, Map<String, Number>> UserPoolEntry : raw.entrySet()) {
+                String user = UserPoolEntry.getKey();
+                ret.put(user, new HashMap<String, Double>());
+                for(Map.Entry<String, Number> resourceEntry : UserPoolEntry.getValue().entrySet()) {
+                    ret.get(user).put(resourceEntry.getKey(), resourceEntry.getValue().doubleValue());
+                }
             }
-            for (Map.Entry<String, Map<WorkerSlot, Collection<ExecutorDetails>>> topo_sched : schedulingMap.get(entry.getKey()).entrySet()) {
-                str.append("\t-->Topology: " + topo_sched.getKey() + "\n");
-                for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> ws : topo_sched.getValue().entrySet()) {
-                    str.append("\t\t->Slot [" + ws.getKey().getPort() + "] -> " + ws.getValue() + "\n");
+        }
+
+        Map fromFile = Utils.findAndReadConfigFile("user-resource-pools.yaml", false);
+        Map<String, Map<String, Number>>tmp = (Map<String, Map<String, Number>>)fromFile.get(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS);
+        if (tmp != null) {
+            for(Map.Entry<String, Map<String, Number>> UserPoolEntry : tmp.entrySet()) {
+                String user = UserPoolEntry.getKey();
+                ret.put(user, new HashMap<String, Double>());
+                for(Map.Entry<String, Number> resourceEntry : UserPoolEntry.getValue().entrySet()) {
+                    ret.get(user).put(resourceEntry.getKey(), resourceEntry.getValue().doubleValue());
                 }
             }
         }
-        return str.toString();
+        return ret;
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/0d34abf9/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java
index c22d5bc..8e11384 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java
@@ -22,13 +22,20 @@ import backtype.storm.Config;
 import backtype.storm.generated.Bolt;
 import backtype.storm.generated.SpoutSpec;
 import backtype.storm.generated.StormTopology;
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.ExecutorDetails;
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.scheduler.WorkerSlot;
 import org.json.simple.JSONObject;
 import org.json.simple.parser.JSONParser;
 import org.json.simple.parser.ParseException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Collection;
 import java.util.HashMap;
+import java.util.LinkedList;
 import java.util.Map;
 
 public class ResourceUtils {
@@ -130,4 +137,48 @@ public class ResourceUtils {
                     Com, topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT));
         }
     }
+
+    /**
+     * print scheduling for debug purposes
+     * @param cluster
+     * @param topologies
+     */
+    public static String printScheduling(Cluster cluster, Topologies topologies) {
+        StringBuilder str = new StringBuilder();
+        Map<String, Map<String, Map<WorkerSlot, Collection<ExecutorDetails>>>> schedulingMap = new HashMap<String, Map<String, Map<WorkerSlot, Collection<ExecutorDetails>>>>();
+        for (TopologyDetails topo : topologies.getTopologies()) {
+            if (cluster.getAssignmentById(topo.getId()) != null) {
+                for (Map.Entry<ExecutorDetails, WorkerSlot> entry : cluster.getAssignmentById(topo.getId()).getExecutorToSlot().entrySet()) {
+                    WorkerSlot slot = entry.getValue();
+                    String nodeId = slot.getNodeId();
+                    ExecutorDetails exec = entry.getKey();
+                    if (schedulingMap.containsKey(nodeId) == false) {
+                        schedulingMap.put(nodeId, new HashMap<String, Map<WorkerSlot, Collection<ExecutorDetails>>>());
+                    }
+                    if (schedulingMap.get(nodeId).containsKey(topo.getId()) == false) {
+                        schedulingMap.get(nodeId).put(topo.getId(), new HashMap<WorkerSlot, Collection<ExecutorDetails>>());
+                    }
+                    if (schedulingMap.get(nodeId).get(topo.getId()).containsKey(slot) == false) {
+                        schedulingMap.get(nodeId).get(topo.getId()).put(slot, new LinkedList<ExecutorDetails>());
+                    }
+                    schedulingMap.get(nodeId).get(topo.getId()).get(slot).add(exec);
+                }
+            }
+        }
+
+        for (Map.Entry<String, Map<String, Map<WorkerSlot, Collection<ExecutorDetails>>>> entry : schedulingMap.entrySet()) {
+            if (cluster.getSupervisorById(entry.getKey()) != null) {
+                str.append("/** Node: " + cluster.getSupervisorById(entry.getKey()).getHost() + "-" + entry.getKey() + " **/\n");
+            } else {
+                str.append("/** Node: Unknown may be dead -" + entry.getKey() + " **/\n");
+            }
+            for (Map.Entry<String, Map<WorkerSlot, Collection<ExecutorDetails>>> topo_sched : schedulingMap.get(entry.getKey()).entrySet()) {
+                str.append("\t-->Topology: " + topo_sched.getKey() + "\n");
+                for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> ws : topo_sched.getValue().entrySet()) {
+                    str.append("\t\t->Slot [" + ws.getKey().getPort() + "] -> " + ws.getValue() + "\n");
+                }
+            }
+        }
+        return str.toString();
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/0d34abf9/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
new file mode 100644
index 0000000..32af686
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
@@ -0,0 +1,239 @@
+/**
+ * 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 backtype.storm.scheduler.resource;
+
+import backtype.storm.scheduler.TopologyDetails;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+public class User {
+    private String userId;
+    //Topologies yet to be scheduled sorted by priority for each user
+    private Set<TopologyDetails> pendingQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
+
+    //Topologies yet to be scheduled sorted by priority for each user
+    private Set<TopologyDetails> runningQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
+
+    //Topologies that was attempted to be scheduled but wasn't successull
+    private Set<TopologyDetails> attemptedQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
+
+    private Map<String, Double> resourcePool = new HashMap<String, Double>();
+
+    private static final Logger LOG = LoggerFactory.getLogger(User.class);
+
+    public User(String userId) {
+        this.userId = userId;
+    }
+
+    public User(String userId, Map<String, Double> resourcePool) {
+        this(userId);
+        if (resourcePool != null) {
+            this.resourcePool.putAll(resourcePool);
+        }
+    }
+
+    public String getId() {
+        return this.userId;
+    }
+
+    public void addTopologyToPendingQueue(TopologyDetails topo) {
+        this.pendingQueue.add(topo);
+    }
+
+    public void addTopologyToRunningQueue(TopologyDetails topo) {
+        this.runningQueue.add(topo);
+    }
+
+    public Set<TopologyDetails> getTopologiesPending() {
+        TreeSet<TopologyDetails> ret = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
+        ret.addAll(this.pendingQueue);
+        return ret;
+    }
+
+    public Set<TopologyDetails> getTopologiesRunning() {
+        TreeSet<TopologyDetails> ret = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
+        ret.addAll(this.runningQueue);
+        return ret;
+    }
+
+    public Set<TopologyDetails> getTopologiesAttempted() {
+        TreeSet<TopologyDetails> ret = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
+        ret.addAll(this.attemptedQueue);
+        return ret;
+    }
+
+    public Map<String, Number> getResourcePool() {
+        if (this.resourcePool != null) {
+            return new HashMap<String, Number>(this.resourcePool);
+        }
+        return null;
+    }
+
+    public void moveTopoFromPendingToRunning(TopologyDetails topo) {
+        moveTopology(topo, this.pendingQueue, "pending", this.runningQueue, "running");
+    }
+
+    public void moveTopoFromPendingToAttempted(TopologyDetails topo) {
+        moveTopology(topo, this.pendingQueue, "pending", this.attemptedQueue, "attempted");
+    }
+
+    private void moveTopology(TopologyDetails topo, Set<TopologyDetails> src, String srcName, Set<TopologyDetails> dest, String destName)  {
+        if(topo == null) {
+            return;
+        }
+        if(!src.contains(topo)) {
+            LOG.warn("Topo {} not in User: {} {} queue!", topo.getName(), this.userId, srcName);
+            return;
+        }
+        if(dest.contains(topo)) {
+            LOG.warn("Topo {} already in in User: {} {} queue!", topo.getName(), this.userId, destName);
+            return;
+        }
+        src.remove(topo);
+        dest.add(topo);
+    }
+
+
+    public Double getResourcePoolAverageUtilization() {
+        List<Double> resourceUilitzationList = new LinkedList<Double>();
+        Double cpuResourcePoolUtilization = this.getCPUResourcePoolUtilization();
+        Double memoryResourcePoolUtilization = this.getMemoryResourcePoolUtilization();
+
+        if(cpuResourcePoolUtilization != null && memoryResourcePoolUtilization != null) {
+            return (cpuResourcePoolUtilization + memoryResourcePoolUtilization ) / 2.0;
+        }
+        return Double.MAX_VALUE;
+    }
+
+    public Double getCPUResourcePoolUtilization() {
+        Double cpuGuarantee = this.resourcePool.get("cpu");
+        if (cpuGuarantee != null) {
+            return this.getCPUResourceUsedByUser() / cpuGuarantee;
+        }
+        return null;
+    }
+
+    public Double getMemoryResourcePoolUtilization() {
+        Double memoryGuarantee = this.resourcePool.get("memory");
+        if (memoryGuarantee != null) {
+            return this.getMemoryResourceUsedByUser() / memoryGuarantee;
+        }
+        return null;
+    }
+
+
+    public Double getCPUResourceUsedByUser() {
+        Double sum = 0.0;
+        for (TopologyDetails topo : this.runningQueue) {
+            sum += topo.getTotalRequestedCpu();
+        }
+        return sum;
+    }
+
+    public Double getMemoryResourceUsedByUser() {
+        Double sum = 0.0;
+        for (TopologyDetails topo : this.runningQueue) {
+            sum += topo.getTotalRequestedMemOnHeap() + topo.getTotalRequestedMemOffHeap();
+        }
+        return sum;
+    }
+
+    public Double getMemoryResourceGuaranteed() {
+        return this.resourcePool.get("memory");
+    }
+
+    public Double getCPUResourceGuaranteed() {
+        return this.resourcePool.get("cpu");
+    }
+
+    public TopologyDetails getNextTopologyToSchedule() {
+        for (TopologyDetails topo : this.pendingQueue) {
+            if(!this.attemptedQueue.contains(topo)) {
+                return topo;
+            }
+        }
+        return null;
+    }
+
+    public boolean hasTopologyNeedSchedule() {
+        return (!this.pendingQueue.isEmpty() && (this.pendingQueue.size() - this.attemptedQueue.size()) > 0);
+    }
+
+    @Override
+    public int hashCode() {
+        return this.userId.hashCode();
+    }
+
+    @Override
+    public String toString() {
+        return this.userId;
+    }
+
+    public String getDetailedInfo() {
+        String ret = "\nUser: " + this.userId;
+        ret += "\n - " + " Resource Pool: " + this.resourcePool;
+        ret += "\n - " + " Running Queue: " + this.runningQueue;
+        ret += "\n - " + " Pending Queue: " + this.pendingQueue;
+        ret += "\n - " + " Attempted Queue: " + this.attemptedQueue;
+        ret += "\n - " + " CPU Used: " + this.getCPUResourceUsedByUser() + " CPU guaranteed: " + this.getCPUResourceGuaranteed();
+        ret += "\n - " + " Memory Used: " + this.getMemoryResourceUsedByUser() + " Memory guaranteed: " + this.getMemoryResourceGuaranteed();
+        ret += "\n - " + " % Resource Guarantee Used: \n -- CPU: " + this.getCPUResourcePoolUtilization()
+                + " Memory: " + this.getMemoryResourcePoolUtilization() + " Average: " + this.getResourcePoolAverageUtilization();
+        return ret;
+    }
+
+    public static String getResourcePoolAverageUtilizationForUsers(Collection<User> users) {
+        String ret = "";
+        for(User user : users) {
+            ret += user.getId() + " - " + user.getResourcePoolAverageUtilization() + " ";
+        }
+        return ret;
+    }
+
+    /**
+     * Comparator that sorts topologies by priority and then by submission time
+     */
+    static class PQsortByPriorityAndSubmittionTime implements Comparator<TopologyDetails> {
+
+        public int compare(TopologyDetails topo1, TopologyDetails topo2) {
+            if (topo1.getTopologyPriority() > topo2.getTopologyPriority()) {
+                return 1;
+            } else if (topo1.getTopologyPriority() < topo2.getTopologyPriority()) {
+                return -1;
+            } else {
+                if (topo1.getUpTime() > topo2.getUpTime()) {
+                    return -1;
+                } else if (topo1.getUpTime() < topo2.getUpTime()) {
+                    return 1;
+                } else {
+                    return 0;
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/0d34abf9/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java b/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
index 49e569f..52c4ed1 100644
--- a/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
+++ b/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
@@ -489,17 +489,37 @@ public class ConfigValidation {
     public static class PacemakerAuthTypeValidator extends Validator {
         @Override
         public void validateField(String name, Object o) {
-            if(o == null) {
-                throw new IllegalArgumentException( "Field " + name + " must be set.");
+            if (o == null) {
+                throw new IllegalArgumentException("Field " + name + " must be set.");
             }
 
-            if(o instanceof String &&
-                    (((String)o).equals("NONE") ||
-                            ((String)o).equals("DIGEST") ||
-                            ((String)o).equals("KERBEROS"))) {
+            if (o instanceof String &&
+                    (((String) o).equals("NONE") ||
+                            ((String) o).equals("DIGEST") ||
+                            ((String) o).equals("KERBEROS"))) {
                 return;
             }
-            throw new IllegalArgumentException( "Field " + name + " must be one of \"NONE\", \"DIGEST\", or \"KERBEROS\"");
+            throw new IllegalArgumentException("Field " + name + " must be one of \"NONE\", \"DIGEST\", or \"KERBEROS\"");
+        }
+    }
+
+    public static class UserResourcePoolEntryValidator extends Validator {
+
+        @Override
+        public void validateField(String name, Object o) {
+            if(o == null) {
+                return;
+            }
+            SimpleTypeValidator.validateField(name, Map.class, o);
+            if(!((Map) o).containsKey("cpu") ) {
+                throw new IllegalArgumentException( "Field " + name + " must have map entry with key: cpu");
+            }
+            if(!((Map) o).containsKey("memory") ) {
+                throw new IllegalArgumentException("Field " + name + " must have map entry with key: memory");
+            }
+
+            SimpleTypeValidator.validateField(name, Number.class, ((Map) o).get("cpu"));
+            SimpleTypeValidator.validateField(name, Number.class, ((Map) o).get("memory"));
         }
     }
 


[10/23] storm git commit: first initial implementation

Posted by da...@apache.org.
first initial implementation


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

Branch: refs/heads/master
Commit: 3e8322053c3d7fc7877b2985bae1ad657562c931
Parents: 4cd5efa
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Wed Nov 25 17:13:44 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Fri Dec 4 13:07:11 2015 -0600

----------------------------------------------------------------------
 conf/user-resource-pools-example.yaml           |  26 +
 .../starter/ResourceAwareExampleTopology.java   |   6 +
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   5 +-
 storm-core/src/jvm/backtype/storm/Config.java   |   5 +-
 .../jvm/backtype/storm/scheduler/Cluster.java   |  41 +-
 .../storm/scheduler/SupervisorDetails.java      |   4 +-
 .../backtype/storm/scheduler/Topologies.java    |   8 +-
 .../storm/scheduler/TopologyDetails.java        |  47 +-
 .../storm/scheduler/resource/RAS_Node.java      |  32 +-
 .../storm/scheduler/resource/RAS_Nodes.java     |   7 +-
 .../resource/ResourceAwareScheduler.java        | 228 +++++----
 .../backtype/storm/scheduler/resource/User.java |  52 +-
 .../eviction/DefaultEvictionStrategy.java       |  46 +-
 .../DefaultSchedulingPriorityStrategy.java      |  14 +-
 .../priority/ISchedulingPriorityStrategy.java   |   2 +-
 .../DefaultResourceAwareStrategy.java           |   4 +-
 .../strategies/scheduling/IStrategy.java        |  11 +
 .../storm/validation/ConfigValidation.java      |  12 +-
 .../scheduler/resource_aware_scheduler_test.clj | 129 +++--
 .../storm/scheduler/resource/Experiment.java    | 222 ---------
 .../resource/TestResourceAwareScheduler.java    | 479 ++++++++++++++++++-
 .../storm/scheduler/resource/TestUser.java      |   4 +-
 .../TestUtilsForResourceAwareScheduler.java     |  42 +-
 23 files changed, 893 insertions(+), 533 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/conf/user-resource-pools-example.yaml
----------------------------------------------------------------------
diff --git a/conf/user-resource-pools-example.yaml b/conf/user-resource-pools-example.yaml
new file mode 100644
index 0000000..829a6be
--- /dev/null
+++ b/conf/user-resource-pools-example.yaml
@@ -0,0 +1,26 @@
+# 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.
+
+resource.aware.scheduler.user.pools:
+    jerry:
+        cpu: 1000
+        memory: 8192.0
+    derek:
+        cpu: 10000.0
+        memory: 32768
+    bobby:
+        cpu: 5000.0
+        memory: 16384.0
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java b/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
index e8225d4..a9ac659 100644
--- a/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
+++ b/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
@@ -84,6 +84,12 @@ public class ResourceAwareExampleTopology {
      */
     conf.setTopologyWorkerMaxHeapSize(1024.0);
 
+    // Set topology priority 0-30 with 0 being the highest priority and 30 being the lowest priority.
+    conf.setTopologyPriority(30);
+
+    //Set strategy to schedule topology. If not specified, default to backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy
+    conf.setTopologyStrategy(backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class);
+
     if (args != null && args.length > 0) {
       conf.setNumWorkers(3);
 

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index e22161b..d76825d 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -847,13 +847,14 @@
 
         supervisors (read-all-supervisor-details nimbus all-scheduling-slots supervisor->dead-ports)
         cluster (Cluster. (:inimbus nimbus) supervisors topology->scheduler-assignment conf)
-
+        _ (.setStatusMap cluster (deref (:id->sched-status nimbus)))
         ;; call scheduler.schedule to schedule all the topologies
         ;; the new assignments for all the topologies are in the cluster object.
         _ (.schedule (:scheduler nimbus) topologies cluster)
         _ (.setResourcesMap cluster @(:id->resources nimbus))
         _ (if-not (conf SCHEDULER-DISPLAY-RESOURCE) (.updateAssignedMemoryForTopologyAndSupervisor cluster topologies))
-        _ (reset! (:id->sched-status nimbus) (.getStatusMap cluster))
+        ;;merge with existing statuses
+        _ (reset! (:id->sched-status nimbus) (merge (deref (:id->sched-status nimbus)) (.getStatusMap cluster)))
         _ (reset! (:node-id->resources nimbus) (.getSupervisorsResourcesMap cluster))
         _ (reset! (:id->resources nimbus) (.getResourcesMap cluster))]
     (.getAssignments cluster)))

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index 54af6fa..f3c8c4a 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -2228,11 +2228,10 @@ public class Config extends HashMap<String, Object> {
     }
 
     /**
-     * Takes as input the scheduler class name.
-     * Currently only the Multitenant Scheduler and Resource Aware Scheduler are supported
+     * Takes as input the strategy class name. Strategy must implement the IStrategy interface
      */
     public void setTopologyStrategy(Class<? extends IStrategy> clazz) {
-        if(clazz != null) {
+        if (clazz != null) {
             this.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, clazz.getName());
         }
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
index 2676af1..c35dbbd 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
@@ -28,6 +28,8 @@ import java.util.Set;
 import backtype.storm.Config;
 import backtype.storm.networktopography.DNSToSwitchMapping;
 import backtype.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class Cluster {
 
@@ -91,9 +93,18 @@ public class Cluster {
         this.conf = storm_conf;
     }
 
+    /**
+     * Get a copy of this cluster object
+     */
     public Cluster getCopy() {
-        Cluster copy = new Cluster(this.inimbus, this.supervisors, this.assignments, this.conf);
-        for(Map.Entry<String, String> entry : this.status.entrySet()) {
+        HashMap<String, SchedulerAssignmentImpl> newAssignments = new HashMap<String, SchedulerAssignmentImpl>();
+        for (Map.Entry<String, SchedulerAssignmentImpl> entry : this.assignments.entrySet()) {
+            newAssignments.put(entry.getKey(), new SchedulerAssignmentImpl(entry.getValue().getTopologyId(), entry.getValue().getExecutorToSlot()));
+        }
+        Map newConf = new HashMap<String, Object>();
+        newConf.putAll(this.conf);
+        Cluster copy = new Cluster(this.inimbus, this.supervisors, newAssignments, newConf);
+        for (Map.Entry<String, String> entry : this.status.entrySet()) {
             copy.setStatus(entry.getKey(), entry.getValue());
         }
         return copy;
@@ -447,9 +458,12 @@ public class Cluster {
         return ret;
     }
 
-    public void setAssignments(Map<String, SchedulerAssignment> assignments) {
+    /**
+     * set assignments for cluster
+     */
+    public void setAssignments(Map<String, SchedulerAssignment> newAssignments) {
         this.assignments = new HashMap<String, SchedulerAssignmentImpl>();
-        for(Map.Entry<String, SchedulerAssignmentImpl> entry : this.assignments.entrySet()) {
+        for (Map.Entry<String, SchedulerAssignment> entry : newAssignments.entrySet()) {
             this.assignments.put(entry.getKey(), new SchedulerAssignmentImpl(entry.getValue().getTopologyId(), entry.getValue().getExecutorToSlot()));
         }
     }
@@ -466,7 +480,7 @@ public class Cluster {
      */
     public double getClusterTotalCPUResource() {
         double sum = 0.0;
-        for(SupervisorDetails sup: this.supervisors.values()) {
+        for (SupervisorDetails sup : this.supervisors.values()) {
             sum += sup.getTotalCPU();
         }
         return sum;
@@ -477,7 +491,7 @@ public class Cluster {
      */
     public double getClusterTotalMemoryResource() {
         double sum = 0.0;
-        for(SupervisorDetails sup: this.supervisors.values()) {
+        for (SupervisorDetails sup : this.supervisors.values()) {
             sum += sup.getTotalMemory();
         }
         return sum;
@@ -607,14 +621,29 @@ public class Cluster {
         }
     }
 
+    /**
+     * set scheduler status for a topology
+     */
+    private static final Logger LOG = LoggerFactory
+            .getLogger(Cluster.class);
     public void setStatus(String topologyId, String status) {
         this.status.put(topologyId, status);
     }
 
+    /**
+     * Get all schedule statuses
+     */
     public Map<String, String> getStatusMap() {
         return this.status;
     }
 
+    /**
+     * set scheduler status map
+     */
+    public void setStatusMap(Map<String, String> statusMap) {
+        this.status.putAll(statusMap);
+    }
+
     public void setResources(String topologyId, Double[] resources) {
         this.resources.put(topologyId, resources);
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
index d93252f..afdabe8 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
@@ -120,7 +120,7 @@ public class SupervisorDetails {
 
     public Double getTotalMemory() {
         Double totalMemory = getTotalResource(Config.SUPERVISOR_MEMORY_CAPACITY_MB);
-        if(totalMemory == null) {
+        if (totalMemory == null) {
             throw new IllegalStateException("default value for supervisor.memory.capacity.mb is not set!");
         }
         return totalMemory;
@@ -128,7 +128,7 @@ public class SupervisorDetails {
 
     public Double getTotalCPU() {
         Double totalCPU = getTotalResource(Config.SUPERVISOR_CPU_CAPACITY);
-        if(totalCPU == null) {
+        if (totalCPU == null) {
             throw new IllegalStateException("default value for supervisor.cpu.capacity is not set!");
         }
         return totalCPU;

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
index 59a53c8..3a6361f 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
@@ -29,10 +29,10 @@ public class Topologies {
     Map<String, Map<String, Component>> _allComponents;
 
     public Topologies(Map<String, TopologyDetails> topologies) {
-        if(topologies==null) topologies = new HashMap();
-        this.topologies = new HashMap<String, TopologyDetails>(topologies.size());
+        if(topologies==null) topologies = new HashMap<>();
+        this.topologies = new HashMap<>(topologies.size());
         this.topologies.putAll(topologies);
-        this.nameToId = new HashMap<String, String>(topologies.size());
+        this.nameToId = new HashMap<>(topologies.size());
         
         for (Map.Entry<String, TopologyDetails> entry : topologies.entrySet()) {
             TopologyDetails topology = entry.getValue();
@@ -75,7 +75,7 @@ public class Topologies {
     @Override
     public String toString() {
         String ret = "Topologies:\n";
-        for(TopologyDetails td : this.getTopologies()) {
+        for (TopologyDetails td : this.getTopologies()) {
             ret += td.toString() + "\n";
         }
         return ret;

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
index 3931b43..871ae9b 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
@@ -39,18 +39,18 @@ import org.slf4j.LoggerFactory;
 
 
 public class TopologyDetails {
-    String topologyId;
-    Map topologyConf;
-    StormTopology topology;
-    Map<ExecutorDetails, String> executorToComponent;
-    int numWorkers;
+    private String topologyId;
+    private Map topologyConf;
+    private StormTopology topology;
+    private Map<ExecutorDetails, String> executorToComponent;
+    private int numWorkers;
     //<ExecutorDetails - Task, Map<String - Type of resource, Map<String - type of that resource, Double - amount>>>
     private Map<ExecutorDetails, Map<String, Double>> _resourceList;
     //Max heap size for a worker used by topology
     private Double topologyWorkerMaxHeapSize;
-
+    //topology priority
     private Integer topologyPriority;
-
+    //when topology was launched
     private int launchTime;
 
     private static final Logger LOG = LoggerFactory.getLogger(TopologyDetails.class);
@@ -415,17 +415,16 @@ public class TopologyDetails {
      * Add default resource requirements for a executor
      */
     public void addDefaultResforExec(ExecutorDetails exec) {
-
         Double topologyComponentCpuPcorePercent = Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT), null);
-        if(topologyComponentCpuPcorePercent == null) {
+        if (topologyComponentCpuPcorePercent == null) {
             LOG.warn("default value for topology.component.cpu.pcore.percent needs to be set!");
         }
-        Double topologyComponentResourcesOffheapMemoryMb =  Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB), null);
-        if(topologyComponentResourcesOffheapMemoryMb == null) {
+        Double topologyComponentResourcesOffheapMemoryMb = Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB), null);
+        if (topologyComponentResourcesOffheapMemoryMb == null) {
             LOG.warn("default value for topology.component.resources.offheap.memory.mb needs to be set!");
         }
         Double topologyComponentResourcesOnheapMemoryMb = Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB), null);
-        if(topologyComponentResourcesOnheapMemoryMb == null) {
+        if (topologyComponentResourcesOnheapMemoryMb == null) {
             LOG.warn("default value for topology.component.resources.onheap.memory.mb needs to be set!");
         }
 
@@ -446,11 +445,11 @@ public class TopologyDetails {
      */
     private void initConfigs() {
         this.topologyWorkerMaxHeapSize = Utils.getDouble(this.topologyConf.get(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB), null);
-        if(this.topologyWorkerMaxHeapSize == null) {
+        if (this.topologyWorkerMaxHeapSize == null) {
             LOG.warn("default value for topology.worker.max.heap.size.mb needs to be set!");
         }
         this.topologyPriority = Utils.getInt(this.topologyConf.get(Config.TOPOLOGY_PRIORITY), null);
-        if(this.topologyPriority == null) {
+        if (this.topologyPriority == null) {
             LOG.warn("default value for topology.priority needs to be set!");
         }
     }
@@ -463,17 +462,35 @@ public class TopologyDetails {
         return this.topologyWorkerMaxHeapSize;
     }
 
+    /**
+     * Get the user that submitted this topology
+     */
     public String getTopologySubmitter() {
-       return (String)this.topologyConf.get(Config.TOPOLOGY_SUBMITTER_USER);
+        String user = (String) this.topologyConf.get(Config.TOPOLOGY_SUBMITTER_USER);
+        if (user == null || user.equals("")) {
+            LOG.debug("Topology {} submitted by anonymous user", this.getName());
+            user = "anonymous";
+        }
+        return user;
     }
 
+    /**
+     * get teh priority of this topology
+     */
     public int getTopologyPriority() {
        return this.topologyPriority;
     }
+
+    /**
+     * Get the timestamp of when this topology was launched
+     */
     public int getLaunchTime() {
         return this.launchTime;
     }
 
+    /**
+     * Get how long this topology has been executing
+     */
     public int getUpTime() {
         return Time.currentTimeSecs() - this.launchTime;
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
index 21367f0..54775bf 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
@@ -90,7 +90,7 @@ public class RAS_Node {
 
     public Collection<WorkerSlot> getUsedSlots() {
         Collection<WorkerSlot> ret = new LinkedList<WorkerSlot>();
-        for(Collection<WorkerSlot> workers : _topIdToUsedSlots.values()) {
+        for (Collection<WorkerSlot> workers : _topIdToUsedSlots.values()) {
             ret.addAll(workers);
         }
         return ret;
@@ -243,30 +243,28 @@ public class RAS_Node {
 
     public void freeMemory(double amount) {
         _availMemory += amount;
-        LOG.info("freeing {} memory...avail mem: {}", amount, _availMemory);
-        if(_availMemory > this.getTotalMemoryResources()) {
+        LOG.debug("freeing {} memory on node {}...avail mem: {}", amount, this.getHostname(), _availMemory);
+        if (_availMemory > this.getTotalMemoryResources()) {
             LOG.warn("Freeing more memory than there exists!");
         }
     }
 
     public void freeCPU(double amount) {
         _availCPU += amount;
-        LOG.info("freeing {} CPU...avail CPU: {}", amount, _availCPU);
-        if(_availCPU > this.getAvailableCpuResources()) {
+        LOG.debug("freeing {} CPU on node...avail CPU: {}", amount, this.getHostname(), _availCPU);
+        if (_availCPU > this.getAvailableCpuResources()) {
             LOG.warn("Freeing more memory than there exists!");
         }
     }
 
     public double getMemoryUsedByWorker(WorkerSlot ws) {
         TopologyDetails topo = this.findTopologyUsingWorker(ws);
-        LOG.info("Topology {} using worker {}", topo, ws);
-        if(topo == null) {
+        if (topo == null) {
             return 0.0;
         }
         Collection<ExecutorDetails> execs = this.getExecutors(ws, this._cluster);
-        LOG.info("Worker {} has execs: {}", ws, execs);
         double totalMemoryUsed = 0.0;
-        for(ExecutorDetails exec : execs) {
+        for (ExecutorDetails exec : execs) {
             totalMemoryUsed += topo.getTotalMemReqTask(exec);
         }
         return totalMemoryUsed;
@@ -274,26 +272,23 @@ public class RAS_Node {
 
     public double getCpuUsedByWorker(WorkerSlot ws) {
         TopologyDetails topo = this.findTopologyUsingWorker(ws);
-        LOG.info("Topology {} using worker {}", topo, ws);
-        if(topo == null) {
+        if (topo == null) {
             return 0.0;
         }
         Collection<ExecutorDetails> execs = this.getExecutors(ws, this._cluster);
-        LOG.info("Worker {} has execs: {}", ws, execs);
         double totalCpuUsed = 0.0;
-        for(ExecutorDetails exec : execs) {
+        for (ExecutorDetails exec : execs) {
             totalCpuUsed += topo.getTotalCpuReqTask(exec);
         }
         return totalCpuUsed;
     }
 
     public TopologyDetails findTopologyUsingWorker(WorkerSlot ws) {
-        for(Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
-            LOG.info("topoId: {} workers: {}", entry.getKey(), entry.getValue());
+        for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
             String topoId = entry.getKey();
             Set<WorkerSlot> workers = entry.getValue();
             for (WorkerSlot worker : workers) {
-                if(worker.getNodeId().equals(ws.getNodeId()) && worker.getPort() == ws.getPort()) {
+                if (worker.getNodeId().equals(ws.getNodeId()) && worker.getPort() == ws.getPort()) {
                     return _topologies.getById(topoId);
                 }
             }
@@ -376,7 +371,9 @@ public class RAS_Node {
 
     @Override
     public String toString() {
-        return "{Node: " + _sup.getHost() + ", AvailMem: " + _availMemory.toString() + ", AvailCPU: " + _availCPU.toString() + "}";
+        return "{Node: " + ((_sup == null) ? "null (possibly down)" : _sup.getHost())
+                + ", AvailMem: " + ((_availMemory == null) ? "N/A" : _availMemory.toString())
+                + ", AvailCPU: " + ((_availCPU == null) ? "N/A" : _availCPU.toString()) + "}";
     }
 
     public static int countSlotsUsed(String topId, Collection<RAS_Node> nodes) {
@@ -415,7 +412,6 @@ public class RAS_Node {
         return total;
     }
 
-    //This function is only used for logging information
     public static Collection<ExecutorDetails> getExecutors(WorkerSlot ws, Cluster cluster) {
         Collection<ExecutorDetails> retList = new ArrayList<ExecutorDetails>();
         for (Entry<String, SchedulerAssignment> entry : cluster.getAssignments()

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
index 42fc236..5a99bdd 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
@@ -35,16 +35,11 @@ import java.util.Map;
 public class RAS_Nodes {
 
     private Map<String, RAS_Node> nodeMap;
-    private Cluster cluster;
-    private Topologies topologies;
 
     private static final Logger LOG = LoggerFactory.getLogger(RAS_Nodes.class);
 
-
     public RAS_Nodes(Cluster cluster, Topologies topologies) {
         this.nodeMap = getAllNodesFrom(cluster, topologies);
-        this.cluster = cluster;
-        this.topologies = topologies;
     }
 
     public static Map<String, RAS_Node> getAllNodesFrom(Cluster cluster, Topologies topologies) {
@@ -142,7 +137,7 @@ public class RAS_Nodes {
         for (RAS_Node node : nodeMap.values()) {
             for (WorkerSlot ws : node.getUsedSlots()) {
                 if (workerSlots.contains(ws)) {
-                    LOG.info("freeing ws {} on node {}", ws, node);
+                    LOG.debug("freeing ws {} on node {}", ws, node);
                     node.free(ws);
                 }
             }

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
index f5c8354..53672f6 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
@@ -32,8 +32,8 @@ import backtype.storm.scheduler.IScheduler;
 import backtype.storm.scheduler.Topologies;
 import backtype.storm.scheduler.TopologyDetails;
 import backtype.storm.scheduler.WorkerSlot;
-import backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy;
 
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -55,7 +55,7 @@ public class ResourceAwareScheduler implements IScheduler {
         private Map conf = new Config();
 
         public SchedulingState(Map<String, User> userMap, Cluster cluster, Topologies topologies, RAS_Nodes nodes, Map conf) {
-            for(Map.Entry<String, User> userMapEntry : userMap.entrySet()) {
+            for (Map.Entry<String, User> userMapEntry : userMap.entrySet()) {
                 String userId = userMapEntry.getKey();
                 User user = userMapEntry.getValue();
                 this.userMap.put(userId, user.getCopy());
@@ -84,61 +84,51 @@ public class ResourceAwareScheduler implements IScheduler {
     @Override
     public void schedule(Topologies topologies, Cluster cluster) {
         LOG.info("\n\n\nRerunning ResourceAwareScheduler...");
-        LOG.debug(ResourceUtils.printScheduling(cluster, topologies));
-        LOG.info("topologies: {}", topologies);
-
+        //initialize data structures
         this.initialize(topologies, cluster);
-
-        LOG.info("UserMap:\n{}", this.userMap);
+        //logs everything that is currently scheduled and the location at which they are scheduled
+        LOG.info("Cluster scheduling:\n{}", ResourceUtils.printScheduling(cluster, topologies));
+        //logs the resources available/used for every node
+        LOG.info("Nodes:\n{}", this.nodes);
+        //logs the detailed info about each user
         for (User user : this.getUserMap().values()) {
             LOG.info(user.getDetailedInfo());
         }
 
-        for (TopologyDetails topo : topologies.getTopologies()) {
-            LOG.info("topo {} status: {}", topo, cluster.getStatusMap().get(topo.getId()));
-        }
-
-        LOG.info("Nodes:\n{}", this.nodes);
-
-        //LOG.info("getNextUser: {}", this.getNextUser());
-
         ISchedulingPriorityStrategy schedulingPrioritystrategy = null;
         while (true) {
-            LOG.info("/*********** next scheduling iteration **************/");
 
-            if(schedulingPrioritystrategy == null) {
+            if (schedulingPrioritystrategy == null) {
                 try {
                     schedulingPrioritystrategy = (ISchedulingPriorityStrategy) Utils.newInstance((String) this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY));
                 } catch (RuntimeException e) {
-                    LOG.error("failed to create instance of priority strategy: {} with error: {}! No topology eviction will be done.",
+                    LOG.error("failed to create instance of priority strategy: {} with error: {}! No topologies will be scheduled.",
                             this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY), e.getMessage());
                     break;
                 }
             }
-            //need to re prepare since scheduling state might have been restored
-            schedulingPrioritystrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
-            //Call scheduling priority strategy
-            TopologyDetails td = schedulingPrioritystrategy.getNextTopologyToSchedule();
-            if(td == null) {
+            TopologyDetails td = null;
+            try {
+                //need to re prepare since scheduling state might have been restored
+                schedulingPrioritystrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
+                //Call scheduling priority strategy
+                td = schedulingPrioritystrategy.getNextTopologyToSchedule();
+            } catch (Exception e) {
+                LOG.error("Exception thrown when running priority strategy {}. No topologies will be scheduled! Error: {} StackTrack: {}"
+                        , schedulingPrioritystrategy.getClass().getName(), e.getMessage(), Arrays.toString(e.getStackTrace()));
+                break;
+            }
+            if (td == null) {
                 break;
             }
             scheduleTopology(td);
         }
-
-        //since scheduling state might have been restored thus need to set the cluster and topologies.
-        cluster = this.cluster;
-        topologies = this.topologies;
     }
 
     public void scheduleTopology(TopologyDetails td) {
         User topologySubmitter = this.userMap.get(td.getTopologySubmitter());
         if (cluster.getUnassignedExecutors(td).size() > 0) {
-            LOG.info("/********Scheduling topology {} from User {}************/", td.getName(), topologySubmitter);
-            LOG.info("{}", this.userMap.get(td.getTopologySubmitter()).getDetailedInfo());
-            LOG.info("{}", User.getResourcePoolAverageUtilizationForUsers(this.userMap.values()));
-            LOG.info("Nodes:\n{}", this.nodes);
-            LOG.debug("From cluster:\n{}", ResourceUtils.printScheduling(this.cluster, this.topologies));
-            LOG.debug("From Nodes:\n{}", ResourceUtils.printScheduling(this.nodes));
+            LOG.debug("/********Scheduling topology {} from User {}************/", td.getName(), topologySubmitter);
 
             SchedulingState schedulingState = this.checkpointSchedulingState();
             IStrategy RAStrategy = null;
@@ -147,39 +137,59 @@ public class ResourceAwareScheduler implements IScheduler {
             } catch (RuntimeException e) {
                 LOG.error("failed to create instance of IStrategy: {} with error: {}! Topology {} will not be scheduled.",
                         td.getName(), td.getConf().get(Config.TOPOLOGY_SCHEDULER_STRATEGY), e.getMessage());
-                topologySubmitter.moveTopoFromPendingToInvalid(td, this.cluster);
+                this.restoreCheckpointSchedulingState(schedulingState);
+                //since state is restored need the update User topologySubmitter to the new User object in userMap
+                topologySubmitter = this.userMap.get(td.getTopologySubmitter());
+                topologySubmitter.moveTopoFromPendingToInvalid(td);
+                this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling - failed to create instance of topology strategy "
+                        + td.getConf().get(Config.TOPOLOGY_SCHEDULER_STRATEGY) + ". Please check logs for details");
                 return;
             }
             IEvictionStrategy evictionStrategy = null;
             while (true) {
-                //Need to re prepare scheduling strategy with cluster and topologies in case scheduling state was restored
-                RAStrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
-                SchedulingResult result = RAStrategy.schedule(td);
-                LOG.info("scheduling result: {}", result);
-                if (result.isValid()) {
+                SchedulingResult result = null;
+                try {
+                    //Need to re prepare scheduling strategy with cluster and topologies in case scheduling state was restored
+                    RAStrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
+                    result = RAStrategy.schedule(td);
+                } catch (Exception e) {
+                    LOG.error("Exception thrown when running strategy {} to schedule topology {}. Topology will not be scheduled! Error: {} StackTrack: {}"
+                            , RAStrategy.getClass().getName(), td.getName(), e.getMessage(), Arrays.toString(e.getStackTrace()));
+                    this.restoreCheckpointSchedulingState(schedulingState);
+                    //since state is restored need the update User topologySubmitter to the new User object in userMap
+                    topologySubmitter = this.userMap.get(td.getTopologySubmitter());
+                    topologySubmitter.moveTopoFromPendingToInvalid(td);
+                    this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling - Exception thrown when running strategy {}"
+                            + RAStrategy.getClass().getName() + ". Please check logs for details");
+                }
+                LOG.debug("scheduling result: {}", result);
+                if (result != null && result.isValid()) {
                     if (result.isSuccess()) {
                         try {
                             if (mkAssignment(td, result.getSchedulingResultMap())) {
-                                topologySubmitter.moveTopoFromPendingToRunning(td, this.cluster);
+                                topologySubmitter.moveTopoFromPendingToRunning(td);
+                                this.cluster.setStatus(td.getId(), "Running - " + result.getMessage());
                             } else {
                                 this.restoreCheckpointSchedulingState(schedulingState);
                                 //since state is restored need the update User topologySubmitter to the new User object in userMap
                                 topologySubmitter = this.userMap.get(td.getTopologySubmitter());
-                                topologySubmitter.moveTopoFromPendingToAttempted(td, this.cluster);
+                                topologySubmitter.moveTopoFromPendingToAttempted(td);
+                                this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling - Unable to assign executors to nodes. Please check logs for details");
                             }
                         } catch (IllegalStateException ex) {
                             LOG.error(ex.toString());
-                            LOG.error("Unsuccessful in scheduling: IllegalStateException thrown!", td.getId());
-                            this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
+                            LOG.error("Unsuccessful in scheduling - IllegalStateException thrown when attempting to assign executors to nodes. Error: {} StackTrace: {}",
+                                    ex.getClass().getName(), Arrays.toString(ex.getStackTrace()));
                             this.restoreCheckpointSchedulingState(schedulingState);
                             //since state is restored need the update User topologySubmitter to the new User object in userMap
                             topologySubmitter = this.userMap.get(td.getTopologySubmitter());
-                            topologySubmitter.moveTopoFromPendingToAttempted(td, this.cluster);
+                            topologySubmitter.moveTopoFromPendingToAttempted(td);
+                            this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling - IllegalStateException thrown when attempting to assign executors to nodes. Please check log for details.");
                         }
                         break;
                     } else {
                         if (result.getStatus() == SchedulingStatus.FAIL_NOT_ENOUGH_RESOURCES) {
-                            if(evictionStrategy == null) {
+                            if (evictionStrategy == null) {
                                 try {
                                     evictionStrategy = (IEvictionStrategy) Utils.newInstance((String) this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY));
                                 } catch (RuntimeException e) {
@@ -189,16 +199,29 @@ public class ResourceAwareScheduler implements IScheduler {
                                     break;
                                 }
                             }
-                            //need to re prepare since scheduling state might have been restored
-                            evictionStrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
-                            if (!evictionStrategy.makeSpaceForTopo(td)) {
-                                this.cluster.setStatus(td.getId(), result.getErrorMessage());
+                            boolean madeSpace = false;
+                            try {
+                                //need to re prepare since scheduling state might have been restored
+                                evictionStrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
+                                madeSpace = evictionStrategy.makeSpaceForTopo(td);
+                            } catch (Exception e) {
+                                LOG.error("Exception thrown when running eviction strategy {} to schedule topology {}. No evictions will be done! Error: {} StackTrack: {}"
+                                        , evictionStrategy.getClass().getName(), td.getName(), e.getClass().getName(), Arrays.toString(e.getStackTrace()));
                                 this.restoreCheckpointSchedulingState(schedulingState);
                                 //since state is restored need the update User topologySubmitter to the new User object in userMap
                                 topologySubmitter = this.userMap.get(td.getTopologySubmitter());
                                 topologySubmitter.moveTopoFromPendingToAttempted(td);
                                 break;
                             }
+                            if (!madeSpace) {
+                                LOG.debug("Could not make space for topo {} will move to attempted", td);
+                                this.restoreCheckpointSchedulingState(schedulingState);
+                                //since state is restored need the update User topologySubmitter to the new User object in userMap
+                                topologySubmitter = this.userMap.get(td.getTopologySubmitter());
+                                topologySubmitter.moveTopoFromPendingToAttempted(td);
+                                this.cluster.setStatus(td.getId(), "Not enough resources to schedule - " + result.getErrorMessage());
+                                break;
+                            }
                             continue;
                         } else if (result.getStatus() == SchedulingStatus.FAIL_INVALID_TOPOLOGY) {
                             this.restoreCheckpointSchedulingState(schedulingState);
@@ -225,13 +248,14 @@ public class ResourceAwareScheduler implements IScheduler {
             }
         } else {
             LOG.warn("Topology {} is already fully scheduled!", td.getName());
-            topologySubmitter.moveTopoFromPendingToRunning(td, this.cluster);
-            throw new IllegalStateException("illegal");
+            topologySubmitter.moveTopoFromPendingToRunning(td);
+            if (this.cluster.getStatusMap().get(td.getId()) == null || this.cluster.getStatusMap().get(td.getId()).equals("")) {
+                this.cluster.setStatus(td.getId(), "Fully Scheduled");
+            }
         }
     }
 
     private boolean mkAssignment(TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> schedulerAssignmentMap) {
-        LOG.info("making assignments for topology {}", td);
         if (schedulerAssignmentMap != null) {
             double requestedMemOnHeap = td.getTotalRequestedMemOnHeap();
             double requestedMemOffHeap = td.getTotalRequestedMemOffHeap();
@@ -241,7 +265,6 @@ public class ResourceAwareScheduler implements IScheduler {
             double assignedCpu = 0.0;
 
             Set<String> nodesUsed = new HashSet<String>();
-            int assignedWorkers = schedulerAssignmentMap.keySet().size();
             for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> workerToTasksEntry : schedulerAssignmentMap.entrySet()) {
                 WorkerSlot targetSlot = workerToTasksEntry.getKey();
                 Collection<ExecutorDetails> execsNeedScheduling = workerToTasksEntry.getValue();
@@ -256,8 +279,6 @@ public class ResourceAwareScheduler implements IScheduler {
                 assignedMemOffHeap += targetSlot.getAllocatedMemOffHeap();
                 assignedCpu += targetSlot.getAllocatedCpu();
             }
-            LOG.debug("Topology: {} assigned to {} nodes on {} workers", td.getId(), nodesUsed.size(), assignedWorkers);
-            this.cluster.setStatus(td.getId(), "Fully Scheduled");
 
             Double[] resources = {requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
                     assignedMemOnHeap, assignedMemOffHeap, assignedCpu};
@@ -266,17 +287,17 @@ public class ResourceAwareScheduler implements IScheduler {
                     td.getId(), requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
                     assignedMemOnHeap, assignedMemOffHeap, assignedCpu);
             this.cluster.setResources(td.getId(), resources);
+            updateSupervisorsResources(this.cluster, this.topologies);
             return true;
         } else {
             LOG.warn("schedulerAssignmentMap for topo {} is null. This shouldn't happen!", td.getName());
             return false;
         }
-        updateSupervisorsResources(cluster, topologies);
     }
 
     private void updateSupervisorsResources(Cluster cluster, Topologies topologies) {
         Map<String, Double[]> supervisors_resources = new HashMap<String, Double[]>();
-        Map<String, RAS_Node> nodes = RAS_Node.getAllNodesFrom(cluster, topologies);
+        Map<String, RAS_Node> nodes = RAS_Nodes.getAllNodesFrom(cluster, topologies);
         for (Map.Entry<String, RAS_Node> entry : nodes.entrySet()) {
             RAS_Node node = entry.getValue();
             Double totalMem = node.getTotalMemoryResources();
@@ -297,42 +318,6 @@ public class ResourceAwareScheduler implements IScheduler {
         return this.userMap;
     }
 
-//    public User getNextUser() {
-//        Double least = Double.POSITIVE_INFINITY;
-//        User ret = null;
-//        for (User user : this.userMap.values()) {
-//            LOG.info("getNextUser {}", user.getDetailedInfo());
-//            LOG.info("hasTopologyNeedSchedule: {}", user.hasTopologyNeedSchedule());
-//            if (user.hasTopologyNeedSchedule()) {
-//                Double userResourcePoolAverageUtilization = user.getResourcePoolAverageUtilization();
-//                if(ret!=null) {
-//                    LOG.info("current: {}-{} compareUser: {}-{}", ret.getId(), least, user.getId(), userResourcePoolAverageUtilization);
-//                    LOG.info("{} == {}: {}", least, userResourcePoolAverageUtilization, least == userResourcePoolAverageUtilization);
-//                    LOG.info("{} == {}: {}", least, userResourcePoolAverageUtilization, (Math.abs(least - userResourcePoolAverageUtilization) < 0.0001));
-//
-//                }
-//                if (least > userResourcePoolAverageUtilization) {
-//                    ret = user;
-//                    least = userResourcePoolAverageUtilization;
-//                } else if (Math.abs(least - userResourcePoolAverageUtilization) < 0.0001) {
-//                    double currentCpuPercentage = ret.getCPUResourceGuaranteed() / this.cluster.getClusterTotalCPUResource();
-//                    double currentMemoryPercentage = ret.getMemoryResourceGuaranteed() / this.cluster.getClusterTotalMemoryResource();
-//                    double currentAvgPercentage = (currentCpuPercentage + currentMemoryPercentage) / 2.0;
-//
-//                    double userCpuPercentage = user.getCPUResourceGuaranteed() / this.cluster.getClusterTotalCPUResource();
-//                    double userMemoryPercentage = user.getMemoryResourceGuaranteed() / this.cluster.getClusterTotalMemoryResource();
-//                    double userAvgPercentage = (userCpuPercentage + userMemoryPercentage) / 2.0;
-//                    LOG.info("current: {}-{} compareUser: {}-{}", ret.getId(), currentAvgPercentage, user.getId(), userAvgPercentage);
-//                    if (userAvgPercentage > currentAvgPercentage) {
-//                        ret = user;
-//                        least = userResourcePoolAverageUtilization;
-//                    }
-//                }
-//            }
-//        }
-//        return ret;
-//    }
-
     /**
      * Intialize scheduling and running queues
      *
@@ -340,33 +325,40 @@ public class ResourceAwareScheduler implements IScheduler {
      * @param cluster
      */
     private void initUsers(Topologies topologies, Cluster cluster) {
-
         this.userMap = new HashMap<String, User>();
         Map<String, Map<String, Double>> userResourcePools = this.getUserResourcePools();
-        LOG.info("userResourcePools: {}", userResourcePools);
+        LOG.debug("userResourcePools: {}", userResourcePools);
 
         for (TopologyDetails td : topologies.getTopologies()) {
+            //Get user that submitted topology.  If topology submitter is null or empty string, the topologySubmitter
+            //will be set to anonymous
             String topologySubmitter = td.getTopologySubmitter();
-            if (topologySubmitter == null) {
-                LOG.warn("Topology {} submitted by anonymous user", td.getName());
-                topologySubmitter = "anonymous";
+            //additional safety check to make sure that topologySubmitter is going to be a valid value
+            if (topologySubmitter == null || topologySubmitter.equals("")) {
+                LOG.error("Cannot determine user for topology {}.  Will skip scheduling this topology", td.getName());
+                continue;
             }
             if (!this.userMap.containsKey(topologySubmitter)) {
                 this.userMap.put(topologySubmitter, new User(topologySubmitter, userResourcePools.get(topologySubmitter)));
             }
-            if (cluster.getUnassignedExecutors(td).size() >= td.getExecutors().size()) {
-                this.userMap.get(topologySubmitter).addTopologyToPendingQueue(td, cluster);
+            if (cluster.getUnassignedExecutors(td).size() > 0) {
+                LOG.debug("adding td: {} to pending queue", td.getName());
+                this.userMap.get(topologySubmitter).addTopologyToPendingQueue(td);
             } else {
-                this.userMap.get(topologySubmitter).addTopologyToRunningQueue(td, cluster);
+                LOG.debug("adding td: {} to running queue with existing status: {}", td.getName(), cluster.getStatusMap().get(td.getId()));
+                this.userMap.get(topologySubmitter).addTopologyToRunningQueue(td);
+                if (cluster.getStatusMap().get(td.getId()) == null || cluster.getStatusMap().get(td.getId()).equals("")) {
+                    cluster.setStatus(td.getId(), "Fully Scheduled");
+                }
             }
         }
     }
 
     private void initialize(Topologies topologies, Cluster cluster) {
-        initUsers(topologies, cluster);
         this.cluster = cluster;
         this.topologies = topologies;
         this.nodes = new RAS_Nodes(this.cluster, this.topologies);
+        initUsers(topologies, cluster);
     }
 
     /**
@@ -403,30 +395,36 @@ public class ResourceAwareScheduler implements IScheduler {
     }
 
     private SchedulingState checkpointSchedulingState() {
-        LOG.info("checkpointing scheduling state...");
-        LOG.info("/*********Checkpoint************/");
+        LOG.debug("/*********Checkpoint scheduling state************/");
         for (User user : this.getUserMap().values()) {
-            LOG.info(user.getDetailedInfo());
+            LOG.debug(user.getDetailedInfo());
         }
-        LOG.info("/*********End************/");
+        LOG.debug(ResourceUtils.printScheduling(this.cluster, this.topologies));
+        LOG.debug("nodes:\n{}", this.nodes);
+        LOG.debug("/*********End************/");
         return new SchedulingState(this.userMap, this.cluster, this.topologies, this.nodes, this.conf);
     }
 
     private void restoreCheckpointSchedulingState(SchedulingState schedulingState) {
-        LOG.info("restoring scheduling state...");
-        LOG.info("/*********Before************/");
-        for (User user : this.getUserMap().values()) {
-            LOG.info(user.getDetailedInfo());
-        }
-        this.cluster = schedulingState.cluster;
+        LOG.debug("/*********restoring scheduling state************/");
+        //reseting cluster
+        //Cannot simply set this.cluster=schedulingState.cluster since clojure is immutable
+        this.cluster.setAssignments(schedulingState.cluster.getAssignments());
+        this.cluster.setSupervisorsResourcesMap(schedulingState.cluster.getSupervisorsResourcesMap());
+        this.cluster.setStatusMap(schedulingState.cluster.getStatusMap());
+        this.cluster.setResourcesMap(schedulingState.cluster.getResourcesMap());
+        //don't need to explicitly set data structues like Cluster since nothing can really be changed
+        //unless this.topologies is set to another object
         this.topologies = schedulingState.topologies;
         this.conf = schedulingState.conf;
         this.userMap = schedulingState.userMap;
         this.nodes = schedulingState.nodes;
-        LOG.info("/*********After************/");
+
         for (User user : this.getUserMap().values()) {
-            LOG.info(user.getDetailedInfo());
+            LOG.debug(user.getDetailedInfo());
         }
-        LOG.info("/*********End************/");
+        LOG.debug(ResourceUtils.printScheduling(cluster, topologies));
+        LOG.debug("nodes:\n{}", this.nodes);
+        LOG.debug("/*********End************/");
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
index 2d7c79f..8542120 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
@@ -58,20 +58,26 @@ public class User {
         if (resourcePool != null) {
             this.resourcePool.putAll(resourcePool);
         }
+        if (this.resourcePool.get("cpu") == null) {
+            this.resourcePool.put("cpu", 0.0);
+        }
+        if (this.resourcePool.get("memory") == null) {
+            this.resourcePool.put("memory", 0.0);
+        }
     }
 
     public User getCopy() {
         User newUser = new User(this.userId, this.resourcePool);
-        for(TopologyDetails topo :  this.pendingQueue) {
+        for (TopologyDetails topo : this.pendingQueue) {
             newUser.addTopologyToPendingQueue(topo);
         }
-        for(TopologyDetails topo :  this.runningQueue) {
+        for (TopologyDetails topo : this.runningQueue) {
             newUser.addTopologyToRunningQueue(topo);
         }
-        for(TopologyDetails topo :  this.attemptedQueue) {
+        for (TopologyDetails topo : this.attemptedQueue) {
             newUser.addTopologyToAttemptedQueue(topo);
         }
-        for(TopologyDetails topo :  this.invalidQueue) {
+        for (TopologyDetails topo : this.invalidQueue) {
             newUser.addTopologyToInvalidQueue(topo);
         }
         return newUser;
@@ -134,6 +140,7 @@ public class User {
         ret.addAll(this.invalidQueue);
         return ret;
     }
+
     public Map<String, Number> getResourcePool() {
         if (this.resourcePool != null) {
             return new HashMap<String, Number>(this.resourcePool);
@@ -190,7 +197,7 @@ public class User {
 
 
     private void moveTopology(TopologyDetails topo, Set<TopologyDetails> src, String srcName, Set<TopologyDetails> dest, String destName) {
-        LOG.info("For User {} Moving topo {} from {} to {}", this.userId, topo.getName(), srcName, destName);
+        LOG.debug("For User {} Moving topo {} from {} to {}", this.userId, topo.getName(), srcName, destName);
         if (topo == null) {
             return;
         }
@@ -204,49 +211,49 @@ public class User {
         }
         src.remove(topo);
         dest.add(topo);
-        LOG.info("SRC: {}", src);
-        LOG.info("DEST: {}", dest);
     }
 
 
-    public Double getResourcePoolAverageUtilization() {
-        List<Double> resourceUilitzationList = new LinkedList<Double>();
+    public double getResourcePoolAverageUtilization() {
         Double cpuResourcePoolUtilization = this.getCPUResourcePoolUtilization();
         Double memoryResourcePoolUtilization = this.getMemoryResourcePoolUtilization();
 
         if (cpuResourcePoolUtilization != null && memoryResourcePoolUtilization != null) {
-            return (cpuResourcePoolUtilization + memoryResourcePoolUtilization) / 2.0;
+            //cannot be (cpuResourcePoolUtilization + memoryResourcePoolUtilization)/2
+            //since memoryResourcePoolUtilization or cpuResourcePoolUtilization can be Double.MAX_VALUE
+            //Should not return infinity in that case
+            return ((cpuResourcePoolUtilization) / 2.0) + ((memoryResourcePoolUtilization) / 2.0);
         }
         return Double.MAX_VALUE;
     }
 
-    public Double getCPUResourcePoolUtilization() {
+    public double getCPUResourcePoolUtilization() {
         Double cpuGuarantee = this.resourcePool.get("cpu");
-        if (cpuGuarantee != null) {
-            return this.getCPUResourceUsedByUser() / cpuGuarantee;
+        if (cpuGuarantee == null || cpuGuarantee == 0.0) {
+            return Double.MAX_VALUE;
         }
-        return null;
+        return this.getCPUResourceUsedByUser() / cpuGuarantee;
     }
 
-    public Double getMemoryResourcePoolUtilization() {
+    public double getMemoryResourcePoolUtilization() {
         Double memoryGuarantee = this.resourcePool.get("memory");
-        if (memoryGuarantee != null) {
-            return this.getMemoryResourceUsedByUser() / memoryGuarantee;
+        if (memoryGuarantee == null || memoryGuarantee == 0.0) {
+            return Double.MAX_VALUE;
         }
-        return null;
+        return this.getMemoryResourceUsedByUser() / memoryGuarantee;
     }
 
 
-    public Double getCPUResourceUsedByUser() {
-        Double sum = 0.0;
+    public double getCPUResourceUsedByUser() {
+        double sum = 0.0;
         for (TopologyDetails topo : this.runningQueue) {
             sum += topo.getTotalRequestedCpu();
         }
         return sum;
     }
 
-    public Double getMemoryResourceUsedByUser() {
-        Double sum = 0.0;
+    public double getMemoryResourceUsedByUser() {
+        double sum = 0.0;
         for (TopologyDetails topo : this.runningQueue) {
             sum += topo.getTotalRequestedMemOnHeap() + topo.getTotalRequestedMemOffHeap();
         }
@@ -271,7 +278,6 @@ public class User {
     }
 
     public boolean hasTopologyNeedSchedule() {
-        //return (!this.pendingQueue.isEmpty() && (this.pendingQueue.size() - this.attemptedQueue.size()) > 0);
         return (!this.pendingQueue.isEmpty());
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
index 7ca7ac3..f0401ce 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
@@ -50,54 +50,50 @@ public class DefaultEvictionStrategy implements IEvictionStrategy {
 
     @Override
     public boolean makeSpaceForTopo(TopologyDetails td) {
-        LOG.info("attempting to make space for topo {} from user {}", td.getName(), td.getTopologySubmitter());
+        LOG.debug("attempting to make space for topo {} from user {}", td.getName(), td.getTopologySubmitter());
         User submitter = this.userMap.get(td.getTopologySubmitter());
         if (submitter.getCPUResourceGuaranteed() == null || submitter.getMemoryResourceGuaranteed() == null) {
             return false;
         }
-
         double cpuNeeded = td.getTotalRequestedCpu() / submitter.getCPUResourceGuaranteed();
         double memoryNeeded = (td.getTotalRequestedMemOffHeap() + td.getTotalRequestedMemOnHeap()) / submitter.getMemoryResourceGuaranteed();
 
+        User evictUser = this.findUserWithMostResourcesAboveGuarantee();
         //user has enough resource under his or her resource guarantee to schedule topology
         if ((1.0 - submitter.getCPUResourcePoolUtilization()) >= cpuNeeded && (1.0 - submitter.getMemoryResourcePoolUtilization()) >= memoryNeeded) {
-            User evictUser = this.findUserWithMostResourcesAboveGuarantee();
-            if (evictUser == null) {
-                LOG.info("Cannot make space for topology {} from user {}", td.getName(), submitter.getId());
-                submitter.moveTopoFromPendingToAttempted(td, this.cluster);
+            if (evictUser != null) {
 
-                return false;
+                TopologyDetails topologyEvict = evictUser.getRunningTopologyWithLowestPriority();
+                evictTopology(topologyEvict);
+                return true;
             }
-            TopologyDetails topologyEvict = evictUser.getRunningTopologyWithLowestPriority();
-            LOG.info("topology to evict: {}", topologyEvict);
-            evictTopology(topologyEvict);
-
-            LOG.info("Resources After eviction:\n{}", this.nodes);
-
-            return true;
         } else {
-
-            if ((1.0 - submitter.getCPUResourcePoolUtilization()) < cpuNeeded) {
-
+            if (evictUser != null) {
+                if ((evictUser.getResourcePoolAverageUtilization() - 1.0) > (cpuNeeded + (submitter.getResourcePoolAverageUtilization() - 1.0))) {
+                    TopologyDetails topologyEvict = evictUser.getRunningTopologyWithLowestPriority();
+                    evictTopology(topologyEvict);
+                    return true;
+                }
             }
-
-            if ((1.0 - submitter.getMemoryResourcePoolUtilization()) < memoryNeeded) {
-
+        }
+        //See if there is a lower priority topology that can be evicted from the current user
+        for (TopologyDetails topo : submitter.getTopologiesRunning()) {
+            //check to if there is a topology with a lower priority we can evict
+            if (topo.getTopologyPriority() > td.getTopologyPriority()) {
+                evictTopology(topo);
+                return true;
             }
-            return false;
-
         }
+        return false;
     }
 
     private void evictTopology(TopologyDetails topologyEvict) {
         Collection<WorkerSlot> workersToEvict = this.cluster.getUsedSlotsByTopologyId(topologyEvict.getId());
         User submitter = this.userMap.get(topologyEvict.getTopologySubmitter());
 
-        LOG.info("Evicting Topology {} with workers: {}", topologyEvict.getName(), workersToEvict);
-        LOG.debug("From Nodes:\n{}", ResourceUtils.printScheduling(this.nodes));
+        LOG.info("Evicting Topology {} with workers: {} from user {}", topologyEvict.getName(), workersToEvict, topologyEvict.getTopologySubmitter());
         this.nodes.freeSlots(workersToEvict);
         submitter.moveTopoFromRunningToPending(topologyEvict, this.cluster);
-        LOG.info("check if topology unassigned: {}", this.cluster.getUsedSlotsByTopologyId(topologyEvict.getId()));
     }
 
     private User findUserWithMostResourcesAboveGuarantee() {

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java
index 5096fd6..990ccd6 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java
@@ -28,7 +28,7 @@ import org.slf4j.LoggerFactory;
 
 import java.util.Map;
 
-public class DefaultSchedulingPriorityStrategy implements  ISchedulingPriorityStrategy{
+public class DefaultSchedulingPriorityStrategy implements ISchedulingPriorityStrategy {
     private static final Logger LOG = LoggerFactory
             .getLogger(DefaultSchedulingPriorityStrategy.class);
 
@@ -58,20 +58,15 @@ public class DefaultSchedulingPriorityStrategy implements  ISchedulingPrioritySt
         Double least = Double.POSITIVE_INFINITY;
         User ret = null;
         for (User user : this.userMap.values()) {
-            LOG.info("getNextUser {}", user.getDetailedInfo());
-            LOG.info("hasTopologyNeedSchedule: {}", user.hasTopologyNeedSchedule());
             if (user.hasTopologyNeedSchedule()) {
                 Double userResourcePoolAverageUtilization = user.getResourcePoolAverageUtilization();
-                if(ret!=null) {
-                    LOG.info("current: {}-{} compareUser: {}-{}", ret.getId(), least, user.getId(), userResourcePoolAverageUtilization);
-                    LOG.info("{} == {}: {}", least, userResourcePoolAverageUtilization, least == userResourcePoolAverageUtilization);
-                    LOG.info("{} == {}: {}", least, userResourcePoolAverageUtilization, (Math.abs(least - userResourcePoolAverageUtilization) < 0.0001));
 
-                }
                 if (least > userResourcePoolAverageUtilization) {
                     ret = user;
                     least = userResourcePoolAverageUtilization;
-                } else if (Math.abs(least - userResourcePoolAverageUtilization) < 0.0001) {
+                }
+                // if ResourcePoolAverageUtilization is equal to the user that is being compared
+                else if (Math.abs(least - userResourcePoolAverageUtilization) < 0.0001) {
                     double currentCpuPercentage = ret.getCPUResourceGuaranteed() / this.cluster.getClusterTotalCPUResource();
                     double currentMemoryPercentage = ret.getMemoryResourceGuaranteed() / this.cluster.getClusterTotalMemoryResource();
                     double currentAvgPercentage = (currentCpuPercentage + currentMemoryPercentage) / 2.0;
@@ -79,7 +74,6 @@ public class DefaultSchedulingPriorityStrategy implements  ISchedulingPrioritySt
                     double userCpuPercentage = user.getCPUResourceGuaranteed() / this.cluster.getClusterTotalCPUResource();
                     double userMemoryPercentage = user.getMemoryResourceGuaranteed() / this.cluster.getClusterTotalMemoryResource();
                     double userAvgPercentage = (userCpuPercentage + userMemoryPercentage) / 2.0;
-                    LOG.info("current: {}-{} compareUser: {}-{}", ret.getId(), currentAvgPercentage, user.getId(), userAvgPercentage);
                     if (userAvgPercentage > currentAvgPercentage) {
                         ret = user;
                         least = userResourcePoolAverageUtilization;

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/ISchedulingPriorityStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/ISchedulingPriorityStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/ISchedulingPriorityStrategy.java
index 7e92b3d..a5b0ff5 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/ISchedulingPriorityStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/ISchedulingPriorityStrategy.java
@@ -34,7 +34,7 @@ public interface ISchedulingPriorityStrategy {
     public void prepare(Topologies topologies, Cluster cluster, Map<String, User> userMap, RAS_Nodes nodes);
 
     /**
-     *
+     * Gets the next topology to schedule
      * @return return the next topology to schedule.  If there is no topologies left to schedule, return null
      */
     public TopologyDetails getNextTopologyToSchedule();

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/DefaultResourceAwareStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/DefaultResourceAwareStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/DefaultResourceAwareStrategy.java
index 1950858..75cc5eb 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/DefaultResourceAwareStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/DefaultResourceAwareStrategy.java
@@ -170,10 +170,10 @@ public class DefaultResourceAwareStrategy implements IStrategy {
             LOG.error("Not all executors successfully scheduled: {}",
                     executorsNotScheduled);
             schedulerAssignmentMap = null;
-            result = SchedulingResult.failure(SchedulingStatus.FAIL_NOT_ENOUGH_RESOURCES, "Not all executors successfully scheduled: " + executorsNotScheduled);
+            result = SchedulingResult.failure(SchedulingStatus.FAIL_NOT_ENOUGH_RESOURCES, (td.getExecutors().size() - unassignedExecutors.size()) + "/" + td.getExecutors().size() + " executors scheduled");
         } else {
             LOG.debug("All resources successfully scheduled!");
-            result = SchedulingResult.success(schedulerAssignmentMap);
+            result = SchedulingResult.successWithMsg(schedulerAssignmentMap, "Fully Scheduled by DefaultResourceAwareStrategy");
         }
         if (schedulerAssignmentMap == null) {
             LOG.error("Topology {} not successfully scheduled!", td.getId());

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java
index 12e8ff3..bb2e955 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java
@@ -37,7 +37,18 @@ import backtype.storm.scheduler.resource.User;
  */
 public interface IStrategy {
 
+    /**
+     * initialize prior to scheduling
+     */
     public void prepare(Topologies topologies, Cluster cluster, Map<String, User> userMap, RAS_Nodes nodes);
 
+    /**
+     * This method is invoked to calcuate a scheduling for topology td
+     * @param td
+     * @return returns a SchedulingResult object containing SchedulingStatus object to indicate whether scheduling is successful
+     * The strategy must calculate a scheduling in the format of Map<WorkerSlot, Collection<ExecutorDetails>> where the key of
+     * this map is the worker slot that the value (collection of executors) should be assigned to. if a scheduling is calculated successfully,
+     * put the scheduling map in the SchedulingResult object.
+     */
     public SchedulingResult schedule(TopologyDetails td);
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java b/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
index b82a4ec..57c9f40 100644
--- a/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
+++ b/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
@@ -507,14 +507,14 @@ public class ConfigValidation {
 
         @Override
         public void validateField(String name, Object o) {
-            if(o == null) {
+            if (o == null) {
                 return;
             }
             SimpleTypeValidator.validateField(name, Map.class, o);
-            if(!((Map) o).containsKey("cpu") ) {
-                throw new IllegalArgumentException( "Field " + name + " must have map entry with key: cpu");
+            if (!((Map) o).containsKey("cpu")) {
+                throw new IllegalArgumentException("Field " + name + " must have map entry with key: cpu");
             }
-            if(!((Map) o).containsKey("memory") ) {
+            if (!((Map) o).containsKey("memory")) {
                 throw new IllegalArgumentException("Field " + name + " must have map entry with key: memory");
             }
 
@@ -533,13 +533,13 @@ public class ConfigValidation {
 
         @Override
         public void validateField(String name, Object o) {
-            if(o == null) {
+            if (o == null) {
                 return;
             }
             SimpleTypeValidator.validateField(name, String.class, o);
             try {
                 Class objectClass = Class.forName((String) o);
-                if(!this.classImplements.isAssignableFrom(objectClass)) {
+                if (!this.classImplements.isAssignableFrom(objectClass)) {
                     throw new IllegalArgumentException("Field " + name + " with value " + o + " does not implement " + this.classImplements.getName());
                 }
             } catch (ClassNotFoundException e) {


[16/23] storm git commit: edits based on comments

Posted by da...@apache.org.
edits based on comments


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/7676e0d9
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/7676e0d9
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/7676e0d9

Branch: refs/heads/master
Commit: 7676e0d9cfa9efedc0fcb538bc27b28ea9d8b076
Parents: a7f6913
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Wed Dec 16 10:39:25 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Wed Dec 16 10:39:25 2015 -0600

----------------------------------------------------------------------
 .../jvm/backtype/storm/scheduler/SupervisorDetails.java   |  4 ++--
 .../src/jvm/backtype/storm/scheduler/TopologyDetails.java | 10 +++++-----
 2 files changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/7676e0d9/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
index afdabe8..5943ecf 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
@@ -121,7 +121,7 @@ public class SupervisorDetails {
     public Double getTotalMemory() {
         Double totalMemory = getTotalResource(Config.SUPERVISOR_MEMORY_CAPACITY_MB);
         if (totalMemory == null) {
-            throw new IllegalStateException("default value for supervisor.memory.capacity.mb is not set!");
+            throw new IllegalStateException("default value for " + Config.SUPERVISOR_MEMORY_CAPACITY_MB + " is not set!");
         }
         return totalMemory;
     }
@@ -129,7 +129,7 @@ public class SupervisorDetails {
     public Double getTotalCPU() {
         Double totalCPU = getTotalResource(Config.SUPERVISOR_CPU_CAPACITY);
         if (totalCPU == null) {
-            throw new IllegalStateException("default value for supervisor.cpu.capacity is not set!");
+            throw new IllegalStateException("default value for " + Config.SUPERVISOR_CPU_CAPACITY + " is not set!");
         }
         return totalCPU;
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/7676e0d9/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
index 9e35981..357bec7 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
@@ -419,15 +419,15 @@ public class TopologyDetails {
     public void addDefaultResforExec(ExecutorDetails exec) {
         Double topologyComponentCpuPcorePercent = Utils.getDouble(this.topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT), null);
         if (topologyComponentCpuPcorePercent == null) {
-            LOG.warn("default value for topology.component.cpu.pcore.percent needs to be set!");
+            LOG.warn("default value for " + Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT + " needs to be set!");
         }
         Double topologyComponentResourcesOffheapMemoryMb = Utils.getDouble(this.topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB), null);
         if (topologyComponentResourcesOffheapMemoryMb == null) {
-            LOG.warn("default value for topology.component.resources.offheap.memory.mb needs to be set!");
+            LOG.warn("default value for " + Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB + " needs to be set!");
         }
         Double topologyComponentResourcesOnheapMemoryMb = Utils.getDouble(this.topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB), null);
         if (topologyComponentResourcesOnheapMemoryMb == null) {
-            LOG.warn("default value for topology.component.resources.onheap.memory.mb needs to be set!");
+            LOG.warn("default value for " + Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB + " needs to be set!");
         }
 
         Map<String, Double> defaultResourceList = new HashMap<>();
@@ -448,11 +448,11 @@ public class TopologyDetails {
     private void initConfigs() {
         this.topologyWorkerMaxHeapSize = Utils.getDouble(this.topologyConf.get(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB), null);
         if (this.topologyWorkerMaxHeapSize == null) {
-            LOG.warn("default value for topology.worker.max.heap.size.mb needs to be set!");
+            LOG.warn("default value for " + Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB + " needs to be set!");
         }
         this.topologyPriority = Utils.getInt(this.topologyConf.get(Config.TOPOLOGY_PRIORITY), null);
         if (this.topologyPriority == null) {
-            LOG.warn("default value for topology.priority needs to be set!");
+            LOG.warn("default value for " + Config.TOPOLOGY_PRIORITY + " needs to be set!");
         }
     }
 


[07/23] storm git commit: made scheduling, eviction, and priority strategies pluggable

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4cd5efa3/storm-core/test/jvm/backtype/storm/TestConfigValidate.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/TestConfigValidate.java b/storm-core/test/jvm/backtype/storm/TestConfigValidate.java
index a5e06c3..048cc92 100644
--- a/storm-core/test/jvm/backtype/storm/TestConfigValidate.java
+++ b/storm-core/test/jvm/backtype/storm/TestConfigValidate.java
@@ -688,6 +688,31 @@ public class TestConfigValidate {
         }
     }
 
+    @Test
+    public void TestImplementsClassValidator() {
+        TestConfig config = new TestConfig();
+        Collection<Object> passCases = new LinkedList<Object>();
+        Collection<Object> failCases = new LinkedList<Object>();
+
+        passCases.add("backtype.storm.networktopography.DefaultRackDNSToSwitchMapping");
+
+        for (Object value : passCases) {
+            config.put(TestConfig.TEST_MAP_CONFIG_8, value);
+            ConfigValidation.validateFields(config, TestConfig.class);
+        }
+
+        failCases.add("backtype.storm.nimbus.NimbusInfo");
+        failCases.add(null);
+        for (Object value : failCases) {
+            try {
+                config.put(TestConfig.TEST_MAP_CONFIG_8, value);
+                ConfigValidation.validateFields(config, TestConfig.class);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+    }
+
     public class TestConfig extends HashMap<String, Object> {
         @isMapEntryType(keyType = String.class, valueType = Integer.class)
         public static final String TEST_MAP_CONFIG = "test.map.config";
@@ -714,5 +739,9 @@ public class TestConfigValidate {
 
         @isMapEntryCustom(keyValidatorClasses = {StringValidator.class}, valueValidatorClasses = {UserResourcePoolEntryValidator.class})
         public static final String TEST_MAP_CONFIG_7 = "test.map.config.7";
+
+        @isImplementationOfClass(implementsClass = backtype.storm.networktopography.DNSToSwitchMapping.class)
+        @NotNull
+        public static final String TEST_MAP_CONFIG_8 = "test.map.config.8";
     }
 }


[02/23] storm git commit: adding unit tests for STORM-898

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj b/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
index aec297d..4fff405 100644
--- a/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
+++ b/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
@@ -24,7 +24,7 @@
            [backtype.storm.topology TopologyBuilder])
   (:import [backtype.storm.scheduler Cluster SupervisorDetails WorkerSlot ExecutorDetails
             SchedulerAssignmentImpl Topologies TopologyDetails])
-  (:import [backtype.storm.scheduler.resource RAS_Node ResourceAwareScheduler])
+  (:import [backtype.storm.scheduler.resource RAS_Node RAS_Nodes ResourceAwareScheduler])
   (:import [backtype.storm Config StormSubmitter])
   (:import [backtype.storm LocalDRPC LocalCluster])
   (:import [java.util HashMap]))
@@ -92,12 +92,12 @@
                     super (+ cpu (.get super->cpu-usage super))))))  ;; add all topo's cpu usage for each super
     super->cpu-usage))
 
-;; testing resource/Node class
+; testing resource/Node class
 (deftest test-node
   (let [supers (gen-supervisors 5 4)
         cluster (Cluster. (nimbus/standalone-nimbus) supers {} {})
         topologies (Topologies. (to-top-map []))
-        node-map (RAS_Node/getAllNodesFrom cluster topologies)
+        node-map (RAS_Nodes/getAllNodesFrom cluster topologies)
         topology1 (TopologyDetails. "topology1" {} nil 0)
         topology2 (TopologyDetails. "topology2" {} nil 0)]
     (is (= 5 (.size node-map)))
@@ -109,31 +109,31 @@
       (is (= 4 (.totalSlotsFree node)))
       (is (= 0 (.totalSlotsUsed node)))
       (is (= 4 (.totalSlots node)))
-      (.assign node topology1 (list (ExecutorDetails. 1 1)) cluster)
+      (.assign node topology1 (list (ExecutorDetails. 1 1)))
       (is (= 1 (.size (.getRunningTopologies node))))
       (is (= false (.isTotallyFree node)))
       (is (= 3 (.totalSlotsFree node)))
       (is (= 1 (.totalSlotsUsed node)))
       (is (= 4 (.totalSlots node)))
-      (.assign node topology1 (list (ExecutorDetails. 2 2)) cluster)
+      (.assign node topology1 (list (ExecutorDetails. 2 2)))
       (is (= 1 (.size (.getRunningTopologies node))))
       (is (= false (.isTotallyFree node)))
       (is (= 2 (.totalSlotsFree node)))
       (is (= 2 (.totalSlotsUsed node)))
       (is (= 4 (.totalSlots node)))
-      (.assign node topology2 (list (ExecutorDetails. 1 1)) cluster)
+      (.assign node topology2 (list (ExecutorDetails. 1 1)))
       (is (= 2 (.size (.getRunningTopologies node))))
       (is (= false (.isTotallyFree node)))
       (is (= 1 (.totalSlotsFree node)))
       (is (= 3 (.totalSlotsUsed node)))
       (is (= 4 (.totalSlots node)))
-      (.assign node topology2 (list (ExecutorDetails. 2 2)) cluster)
+      (.assign node topology2 (list (ExecutorDetails. 2 2)))
       (is (= 2 (.size (.getRunningTopologies node))))
       (is (= false (.isTotallyFree node)))
       (is (= 0 (.totalSlotsFree node)))
       (is (= 4 (.totalSlotsUsed node)))
       (is (= 4 (.totalSlots node)))
-      (.freeAllSlots node cluster)
+      (.freeAllSlots node)
       (is (= 0 (.size (.getRunningTopologies node))))
       (is (= true (.isTotallyFree node)))
       (is (= 4 (.totalSlotsFree node)))
@@ -153,7 +153,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
-                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
+                     TOPOLOGY-PRIORITY 0}
                     storm-topology
                     1
                     (mk-ed-map [["wordSpout" 0 1]
@@ -162,8 +163,9 @@
                   {STORM-NETWORK-TOPOGRAPHY-PLUGIN
                    "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
         topologies (Topologies. (to-top-map [topology1]))
-        node-map (RAS_Node/getAllNodesFrom cluster topologies)
+        node-map (RAS_Nodes/getAllNodesFrom cluster topologies)
         scheduler (ResourceAwareScheduler.)]
+    (.prepare scheduler {})
     (.schedule scheduler topologies cluster)
     (let [assignment (.getAssignmentById cluster "topology1")
           assigned-slots (.getSlots assignment)
@@ -192,7 +194,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
-                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
+                     TOPOLOGY-PRIORITY 0}
                     storm-topology1
                     1
                     (mk-ed-map [["wordSpout1" 0 1]
@@ -212,7 +215,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
-                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
+                     TOPOLOGY-PRIORITY 0}
                     storm-topology2
                     1
                     (mk-ed-map [["wordSpoutX" 0 1]
@@ -223,6 +227,7 @@
                    "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
         topologies (Topologies. (to-top-map [topology1 topology2]))
         scheduler (ResourceAwareScheduler.)]
+    (.prepare scheduler {})
     (.schedule scheduler topologies cluster)
     (let [assignment (.getAssignmentById cluster "topology1")
           assigned-slots (.getSlots assignment)
@@ -255,7 +260,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
-                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
+                     TOPOLOGY-PRIORITY 0}
                     storm-topology
                     2
                     (mk-ed-map [["wordSpout" 0 1]
@@ -265,6 +271,7 @@
                    "backtype.storm.testing.AlternateRackDNSToSwitchMapping"})
         topologies (Topologies. (to-top-map [topology2]))
         scheduler (ResourceAwareScheduler.)]
+    (.prepare scheduler {})
     (.schedule scheduler topologies cluster)
     (let [assignment (.getAssignmentById cluster "topology2")
           assigned-slots (.getSlots assignment)
@@ -292,7 +299,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
-                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
+                     TOPOLOGY-PRIORITY 0}
                     storm-topology
                     2 ;; need two workers, each on one node
                     (mk-ed-map [["wordSpout" 0 2]
@@ -302,6 +310,7 @@
                    "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
         topologies (Topologies. (to-top-map [topology1]))
         scheduler (ResourceAwareScheduler.)]
+    (.prepare scheduler {})
     (.schedule scheduler topologies cluster)
     (let [assignment (.getAssignmentById cluster "topology1")
           assigned-slots (.getSlots assignment)
@@ -331,7 +340,7 @@
     (is (< (Math/abs (- 100.0 (first assigned-ed-cpu))) epsilon))
     (is (< (Math/abs (- 250.0 (second assigned-ed-cpu))) epsilon))
     (is (< (Math/abs (- 250.0 (last assigned-ed-cpu))) epsilon))
-    (doseq [[avail used] mem-avail->used] ;; for each node, assigned mem smaller than total 
+    (doseq [[avail used] mem-avail->used] ;; for each node, assigned mem smaller than total
       (is (>= avail used)))
     (doseq [[avail used] cpu-avail->used] ;; for each node, assigned cpu smaller than total
       (is (>= avail used))))
@@ -348,7 +357,8 @@
                       TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
                       TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                       TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
-                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
+                      TOPOLOGY-PRIORITY 0}
                      storm-topology1
                      3 ;; three workers to hold three executors
                      (mk-ed-map [["spout1" 0 3]]))
@@ -361,7 +371,8 @@
                       TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 1280.0 ;; large enough thus two eds can not be fully assigned to one node
                       TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                       TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
-                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
+                      TOPOLOGY-PRIORITY 0}
                      storm-topology2
                      2  ;; two workers, each holds one executor and resides on one node
                      (mk-ed-map [["spout2" 0 2]]))
@@ -372,6 +383,7 @@
                               {STORM-NETWORK-TOPOGRAPHY-PLUGIN
                                "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
             topologies (Topologies. (to-top-map [topology2]))
+            _ (.prepare scheduler {})
             _ (.schedule scheduler topologies cluster)
             assignment (.getAssignmentById cluster "topology2")
             failed-worker (first (vec (.getSlots assignment)))  ;; choose a worker to mock as failed
@@ -380,6 +392,7 @@
             _ (doseq [ed failed-eds] (.remove ed->slot ed))  ;; remove executor details assigned to the worker
             copy-old-mapping (HashMap. ed->slot)
             healthy-eds (.keySet copy-old-mapping)
+            _ (.prepare scheduler {})
             _ (.schedule scheduler topologies cluster)
             new-assignment (.getAssignmentById cluster "topology2")
             new-ed->slot (.getExecutorToSlot new-assignment)]
@@ -387,7 +400,7 @@
         (doseq [ed healthy-eds]
           (is (.equals (.get copy-old-mapping ed) (.get new-ed->slot ed))))
         (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))
-    
+
     (testing "When a supervisor fails, RAS does not alter existing assignments"
       (let [existing-assignments {"topology1" (SchedulerAssignmentImpl. "topology1"
                                                                         {(ExecutorDetails. 0 0) (WorkerSlot. "id0" 0)    ;; worker 0 on the failed super
@@ -401,11 +414,12 @@
             ed->slot (.getExecutorToSlot assignment)
             copy-old-mapping (HashMap. ed->slot)
             existing-eds (.keySet copy-old-mapping)  ;; all the three eds on three workers
-            new-cluster (Cluster. (nimbus/standalone-nimbus) 
+            new-cluster (Cluster. (nimbus/standalone-nimbus)
                                   (dissoc supers "id0")        ;; mock the super0 as a failed supervisor
                                   (.getAssignments cluster)
                                   {STORM-NETWORK-TOPOGRAPHY-PLUGIN
                                    "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+            _ (.prepare scheduler {})
             _ (.schedule scheduler topologies new-cluster) ;; the actual schedule for this topo will not run since it is fully assigned
             new-assignment (.getAssignmentById new-cluster "topology1")
             new-ed->slot (.getExecutorToSlot new-assignment)]
@@ -427,11 +441,12 @@
             _ (.remove ed->slot (ExecutorDetails. 1 1))  ;; delete one worker of super0 (failed) from topo1 assignment to enable actual schedule for testing
             copy-old-mapping (HashMap. ed->slot)
             existing-eds (.keySet copy-old-mapping)  ;; namely the two eds on the orphaned worker and the healthy worker
-            new-cluster (Cluster. (nimbus/standalone-nimbus) 
+            new-cluster (Cluster. (nimbus/standalone-nimbus)
                                   (dissoc supers "id0")        ;; mock the super0 as a failed supervisor
                                   (.getAssignments cluster)
                                   {STORM-NETWORK-TOPOGRAPHY-PLUGIN
                                    "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+            _ (.prepare scheduler {})
             _ (.schedule scheduler topologies new-cluster)
             new-assignment (.getAssignmentById new-cluster "topology1")
             new-ed->slot (.getExecutorToSlot new-assignment)]
@@ -444,11 +459,13 @@
                               {STORM-NETWORK-TOPOGRAPHY-PLUGIN
                                "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
             topologies (Topologies. (to-top-map [topology1]))
+            _ (.prepare scheduler {})
             _ (.schedule scheduler topologies cluster)
             assignment (.getAssignmentById cluster "topology1")
             ed->slot (.getExecutorToSlot assignment)
             copy-old-mapping (HashMap. ed->slot)
             new-topologies (Topologies. (to-top-map [topology1 topology2]))  ;; a second topology joins
+            _ (.prepare scheduler {})
             _ (.schedule scheduler new-topologies cluster)
             new-assignment (.getAssignmentById cluster "topology1")
             new-ed->slot (.getExecutorToSlot new-assignment)]
@@ -459,17 +476,17 @@
 
 ;; Automated tests for heterogeneous cluster
 (deftest test-heterogeneous-cluster
-  (let [supers (into {} (for [super [(SupervisorDetails. (str "id" 0) (str "host" 0) (list ) 
+  (let [supers (into {} (for [super [(SupervisorDetails. (str "id" 0) (str "host" 0) (list )
                                                          (map int (list 1 2 3 4))
                                                          {Config/SUPERVISOR_MEMORY_CAPACITY_MB 4096.0
                                                           Config/SUPERVISOR_CPU_CAPACITY 800.0})
-                                     (SupervisorDetails. (str "id" 1) (str "host" 1) (list ) 
+                                     (SupervisorDetails. (str "id" 1) (str "host" 1) (list )
                                                          (map int (list 1 2 3 4))
                                                          {Config/SUPERVISOR_MEMORY_CAPACITY_MB 1024.0
                                                           Config/SUPERVISOR_CPU_CAPACITY 200.0})]]
                           {(.getId super) super}))
         builder1 (TopologyBuilder.)  ;; topo1 has one single huge task that can not be handled by the small-super
-        _ (doto (.setSpout builder1 "spout1" (TestWordSpout.) 1) 
+        _ (doto (.setSpout builder1 "spout1" (TestWordSpout.) 1)
             (.setMemoryLoad 2000.0 48.0)
             (.setCPULoad 300.0))
         storm-topology1 (.createTopology builder1)
@@ -479,7 +496,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
-                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
+                     TOPOLOGY-PRIORITY 0}
                     storm-topology1
                     1
                     (mk-ed-map [["spout1" 0 1]]))
@@ -494,9 +512,10 @@
                      TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
-                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
+                     TOPOLOGY-PRIORITY 0}
                     storm-topology2
-                    2 
+                    2
                     (mk-ed-map [["spout2" 0 4]]))
         builder3 (TopologyBuilder.) ;; topo3 has 4 medium tasks, launching topo 1-3 together requires the same mem as the cluster's mem capacity (5G)
         _ (doto (.setSpout builder3 "spout3" (TestWordSpout.) 4)
@@ -509,9 +528,10 @@
                      TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
-                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
+                     TOPOLOGY-PRIORITY 0}
                     storm-topology3
-                    2 
+                    2
                     (mk-ed-map [["spout3" 0 4]]))
         builder4 (TopologyBuilder.) ;; topo4 has 12 small tasks, each's mem req does not exactly divide a node's mem capacity
         _ (doto (.setSpout builder4 "spout4" (TestWordSpout.) 2)
@@ -524,9 +544,10 @@
                      TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
-                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
+                     TOPOLOGY-PRIORITY 0}
                     storm-topology4
-                    2 
+                    2
                     (mk-ed-map [["spout4" 0 12]]))
         builder5 (TopologyBuilder.) ;; topo5 has 40 small tasks, it should be able to exactly use up both the cpu and mem in teh cluster
         _ (doto (.setSpout builder5 "spout5" (TestWordSpout.) 40)
@@ -539,26 +560,28 @@
                      TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
-                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
+                     TOPOLOGY-PRIORITY 0}
                     storm-topology5
-                    2 
+                    2
                     (mk-ed-map [["spout5" 0 40]]))
         epsilon 0.000001
-        topologies (Topologies. (to-top-map [topology1 topology2]))
-        scheduler (ResourceAwareScheduler.)]
+        topologies (Topologies. (to-top-map [topology1 topology2]))]
 
     (testing "Launch topo 1-3 together, it should be able to use up either mem or cpu resource due to exact division"
       (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
                               {STORM-NETWORK-TOPOGRAPHY-PLUGIN
                                "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
             topologies (Topologies. (to-top-map [topology1 topology2 topology3]))
+            scheduler (ResourceAwareScheduler.)
+            _ (.prepare scheduler {})
             _ (.schedule scheduler topologies cluster)
             super->mem-usage (get-super->mem-usage cluster topologies)
             super->cpu-usage (get-super->cpu-usage cluster topologies)]
         (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
         (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))
         (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology3")))
-        (doseq [super (.values supers)] 
+        (doseq [super (.values supers)]
           (let [mem-avail (.getTotalMemory super)
                 mem-used (.get super->mem-usage super)
                 cpu-avail (.getTotalCPU super)
@@ -571,6 +594,8 @@
                               {STORM-NETWORK-TOPOGRAPHY-PLUGIN
                                "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
             topologies (Topologies. (to-top-map [topology1 topology2 topology3]))
+            scheduler (ResourceAwareScheduler.)
+            _ (.prepare scheduler {})
             _ (.schedule scheduler topologies cluster)
                 scheduled-topos (if (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")) 1 0)
                 scheduled-topos (+ scheduled-topos (if (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")) 1 0))
@@ -582,11 +607,13 @@
                               {STORM-NETWORK-TOPOGRAPHY-PLUGIN
                                "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
             topologies (Topologies. (to-top-map [topology5]))
+            scheduler (ResourceAwareScheduler.)
+            _ (.prepare scheduler {})
             _ (.schedule scheduler topologies cluster)
             super->mem-usage (get-super->mem-usage cluster topologies)
             super->cpu-usage (get-super->cpu-usage cluster topologies)]
         (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology5")))
-        (doseq [super (.values supers)] 
+        (doseq [super (.values supers)]
           (let [mem-avail (.getTotalMemory super)
                 mem-used (.get super->mem-usage super)
                 cpu-avail (.getTotalCPU ^SupervisorDetails super)
@@ -606,15 +633,17 @@
           storm-topology1 (.createTopology builder1)
           topology1 (TopologyDetails. "topology1"
                       {TOPOLOGY-NAME "topology-name-1"
-                       TOPOLOGY-SUBMITTER-USER "userC"
+                       TOPOLOGY-SUBMITTER-USER "userA"
                        TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
                        TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                        TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
-                       TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 128.0}
+                       TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 128.0
+                       TOPOLOGY-PRIORITY 0}
                       storm-topology1
                       1
                       (mk-ed-map [["spout1" 0 4]]))
           topologies (Topologies. (to-top-map [topology1]))]
+      (.prepare scheduler {"userA" {"cpu" 2000.0 "memory" 400.0}})
       (.schedule scheduler topologies cluster)
       (is (= (.get (.getStatusMap cluster) "topology1") "Fully Scheduled"))
       (is (= (.getAssignedNumWorkers cluster topology1) 4)))
@@ -632,18 +661,20 @@
                        TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
                        TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                        TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
-                       TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 128.0}
+                       TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 128.0
+                       TOPOLOGY-PRIORITY 0}
                       storm-topology1
                       1
                       (mk-ed-map [["spout1" 0 5]]))
           topologies (Topologies. (to-top-map [topology1]))]
+      (.prepare scheduler {})
       (.schedule scheduler topologies cluster)
       ;;spout1 is going to contain 5 executors that needs scheduling. Each of those executors has a memory requirement of 128.0 MB
       ;;The cluster contains 4 free WorkerSlots. For this topolology each worker is limited to a max heap size of 128.0
       ;;Thus, one executor not going to be able to get scheduled thus failing the scheduling of this topology and no executors of this topology will be scheduleded
       (is (= (.size (.getUnassignedExecutors cluster topology1)) 5))
-      (is (= (.get (.getStatusMap cluster) "topology1")  "Unsuccessful in scheduling")))
-    
+      (is (= (.get (.getStatusMap cluster) "topology1")  "Not all executors successfully scheduled: [[1, 1]]")))
+
     (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
                              {STORM-NETWORK-TOPOGRAPHY-PLUGIN
                               "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
@@ -652,11 +683,12 @@
           _ (.setSpout builder1 "spout1" (TestWordSpout.) 2)
           storm-topology1 (.createTopology builder1)
           conf  {TOPOLOGY-NAME "topology-name-1"
-                       TOPOLOGY-SUBMITTER-USER "userC"
-                       TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 129.0
-                       TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
-                       TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
-                       TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 128.0}
+                 TOPOLOGY-SUBMITTER-USER "userC"
+                 TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 129.0
+                 TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                 TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                 TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 128.0
+                 TOPOLOGY-PRIORITY 0}
           topology1 (TopologyDetails. "topology1"
                       conf
                       storm-topology1
@@ -665,5 +697,5 @@
           topologies (Topologies. (to-top-map [topology1]))]
       (is (thrown? IllegalArgumentException
             (StormSubmitter/submitTopologyWithProgressBar "test" conf storm-topology1)))
-       
+
   )))

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/test/jvm/backtype/storm/TestConfigValidate.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/TestConfigValidate.java b/storm-core/test/jvm/backtype/storm/TestConfigValidate.java
index 7f193cc..a5e06c3 100644
--- a/storm-core/test/jvm/backtype/storm/TestConfigValidate.java
+++ b/storm-core/test/jvm/backtype/storm/TestConfigValidate.java
@@ -628,9 +628,64 @@ public class TestConfigValidate {
             } catch (IllegalArgumentException Ex) {
             }
         }
+    }
+
+    @Test
+    public void TestResourceAwareSchedulerUserPool() {
+        TestConfig config = new TestConfig();
+        Collection<Object> passCases = new LinkedList<Object>();
+        Collection<Object> failCases = new LinkedList<Object>();
+
+        Map<String, Map<String, Integer>> passCase1 = new HashMap<String, Map<String, Integer>>();
+        passCase1.put("jerry", new HashMap<String, Integer>());
+        passCase1.put("bobby", new HashMap<String, Integer>());
+        passCase1.put("derek", new HashMap<String, Integer>());
+
+        passCase1.get("jerry").put("cpu", 10000);
+        passCase1.get("jerry").put("memory", 20148);
+        passCase1.get("bobby").put("cpu", 20000);
+        passCase1.get("bobby").put("memory", 40148);
+        passCase1.get("derek").put("cpu", 30000);
+        passCase1.get("derek").put("memory", 60148);
+
+        passCases.add(passCase1);
+
+        for (Object value : passCases) {
+            config.put(TestConfig.TEST_MAP_CONFIG_7, value);
+            ConfigValidation.validateFields(config, TestConfig.class);
+        }
+
+        Map<String, Map<String, Integer>> failCase1 = new HashMap<String, Map<String, Integer>>();
+        failCase1.put("jerry", new HashMap<String, Integer>());
+        failCase1.put("bobby", new HashMap<String, Integer>());
+        failCase1.put("derek", new HashMap<String, Integer>());
 
+        failCase1.get("jerry").put("cpu", 10000);
+        failCase1.get("jerry").put("memory", 20148);
+        failCase1.get("bobby").put("cpu", 20000);
+        failCase1.get("bobby").put("memory", 40148);
+        failCase1.get("derek").put("cpu", 30000);
 
+        Map<String, Map<String, Integer>> failCase2 = new HashMap<String, Map<String, Integer>>();
+        failCase2.put("jerry", new HashMap<String, Integer>());
+        failCase2.put("bobby", new HashMap<String, Integer>());
+        failCase2.put("derek", new HashMap<String, Integer>());
+        failCase2.get("bobby").put("cpu", 20000);
+        failCase2.get("bobby").put("memory", 40148);
+        failCase2.get("derek").put("cpu", 30000);
+        failCase2.get("derek").put("memory", 60148);
 
+        failCases.add(failCase1);
+        failCases.add(failCase2);
+
+        for (Object value : failCases) {
+            try {
+                config.put(TestConfig.TEST_MAP_CONFIG_7, value);
+                ConfigValidation.validateFields(config, TestConfig.class);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
     }
 
     public class TestConfig extends HashMap<String, Object> {
@@ -656,5 +711,8 @@ public class TestConfigValidate {
 
         @isMapEntryCustom(keyValidatorClasses = {StringValidator.class}, valueValidatorClasses = {ImpersonationAclUserEntryValidator.class})
         public static final String TEST_MAP_CONFIG_6 = "test.map.config.6";
+
+        @isMapEntryCustom(keyValidatorClasses = {StringValidator.class}, valueValidatorClasses = {UserResourcePoolEntryValidator.class})
+        public static final String TEST_MAP_CONFIG_7 = "test.map.config.7";
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/test/jvm/backtype/storm/scheduler/resource/Experiment.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/Experiment.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/Experiment.java
new file mode 100644
index 0000000..bea41ff
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/scheduler/resource/Experiment.java
@@ -0,0 +1,175 @@
+/**
+ * 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 backtype.storm.scheduler.resource;
+
+import backtype.storm.Config;
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.INimbus;
+import backtype.storm.scheduler.SchedulerAssignmentImpl;
+import backtype.storm.scheduler.SupervisorDetails;
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.utils.Time;
+import backtype.storm.utils.Utils;
+import org.junit.Test;
+import org.junit.Assert;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Created by jerrypeng on 11/11/15.
+ */
+public class Experiment {
+    @Test
+    public void TestMultipleUsers() {
+//        INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
+//        Map<String, Number> resourceMap = new HashMap<String, Number>();
+//        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 1000.0);
+//        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1024.0 * 10);
+//        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(5, 4, resourceMap);
+//        Config config = new Config();
+//        config.putAll(Utils.readDefaultConfig());
+//        Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
+//        resourceUserPool.put("jerry", new HashMap<String, Number>());
+//        resourceUserPool.get("jerry").put("cpu", 1000);
+//        resourceUserPool.get("jerry").put("memory", 8192.0);
+//
+//        resourceUserPool.put("bobby", new HashMap<String, Number>());
+//        resourceUserPool.get("bobby").put("cpu", 10000.0);
+//        resourceUserPool.get("bobby").put("memory", 32768);
+//
+//        resourceUserPool.put("derek", new HashMap<String, Number>());
+//        resourceUserPool.get("derek").put("cpu", 5000.0);
+//        resourceUserPool.get("derek").put("memory", 16384.0);
+//
+//        config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
+//        Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), config);
+//
+//        config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
+//
+//        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
+//        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
+//        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30);
+//        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
+//        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30);
+//
+//        config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
+//
+//        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
+//        TopologyDetails topo7 = TestUtilsForResourceAwareScheduler.getTopology("topo-7", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
+//        TopologyDetails topo8 = TestUtilsForResourceAwareScheduler.getTopology("topo-8", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30);
+//        TopologyDetails topo9 = TestUtilsForResourceAwareScheduler.getTopology("topo-9", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
+//        TopologyDetails topo10 = TestUtilsForResourceAwareScheduler.getTopology("topo-10", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30);
+//
+//        config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
+//
+//        TopologyDetails topo11 = TestUtilsForResourceAwareScheduler.getTopology("topo-11", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
+//        TopologyDetails topo12 = TestUtilsForResourceAwareScheduler.getTopology("topo-12", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
+//        TopologyDetails topo13 = TestUtilsForResourceAwareScheduler.getTopology("topo-13", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30);
+//        TopologyDetails topo14 = TestUtilsForResourceAwareScheduler.getTopology("topo-14", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
+//        TopologyDetails topo15 = TestUtilsForResourceAwareScheduler.getTopology("topo-15", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30);
+//
+//
+//        Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
+//        topoMap.put(topo1.getId(), topo1);
+//        topoMap.put(topo2.getId(), topo2);
+//        topoMap.put(topo3.getId(), topo3);
+//        topoMap.put(topo4.getId(), topo4);
+//        topoMap.put(topo5.getId(), topo5);
+//        topoMap.put(topo6.getId(), topo6);
+//        topoMap.put(topo7.getId(), topo7);
+//        topoMap.put(topo8.getId(), topo8);
+//        topoMap.put(topo9.getId(), topo9);
+//        topoMap.put(topo10.getId(), topo10);
+//        topoMap.put(topo11.getId(), topo11);
+//        topoMap.put(topo12.getId(), topo12);
+//        topoMap.put(topo13.getId(), topo13);
+//        topoMap.put(topo14.getId(), topo14);
+//        topoMap.put(topo15.getId(), topo15);
+//
+//        Topologies topologies = new Topologies(topoMap);
+//
+//        ResourceAwareScheduler rs = new ResourceAwareScheduler();
+//
+//        rs.prepare(config);
+//        rs.schedule(topologies, cluster);
+//
+//        for(TopologyDetails topo : topoMap.values()) {
+//            Assert.assertEquals(cluster.getStatusMap().get(topo.getId()), "Fully Scheduled");
+//        }
+//
+//        for(User user : rs.getUserMap().values()) {
+//            Assert.assertEquals(user.getTopologiesPending().size(), 0);
+//            Assert.assertEquals(user.getTopologiesRunning().size(), 5);
+//        }
+
+        INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
+        Map<String, Number> resourceMap = new HashMap<String, Number>();
+        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 200.0);
+        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1024.0 * 10);
+        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(1, 4, resourceMap);
+        Config config = new Config();
+        config.putAll(Utils.readDefaultConfig());
+        Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
+        resourceUserPool.put("jerry", new HashMap<String, Number>());
+        resourceUserPool.get("jerry").put("cpu", 1000);
+        resourceUserPool.get("jerry").put("memory", 8192.0);
+
+        resourceUserPool.put("bobby", new HashMap<String, Number>());
+        resourceUserPool.get("bobby").put("cpu", 10000.0);
+        resourceUserPool.get("bobby").put("memory", 32768);
+
+        resourceUserPool.put("derek", new HashMap<String, Number>());
+        resourceUserPool.get("derek").put("cpu", 5000.0);
+        resourceUserPool.get("derek").put("memory", 16384.0);
+
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
+        Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), config);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
+
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
+
+        Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
+        topoMap.put(topo1.getId(), topo1);
+        topoMap.put(topo2.getId(), topo2);
+
+        Topologies topologies = new Topologies(topoMap);
+
+        ResourceAwareScheduler rs = new ResourceAwareScheduler();
+
+        rs.prepare(config);
+        rs.schedule(topologies, cluster);
+
+        int fullyScheduled = 0;
+        for (TopologyDetails topo : topoMap.values()) {
+            if(cluster.getStatusMap().get(topo.getId()).equals("Fully Scheduled")) {
+                fullyScheduled++;
+            }
+        }
+        Assert.assertEquals("# of Fully scheduled", 1, fullyScheduled);
+        Assert.assertEquals("# of topologies schedule attempted", 1, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of topologies running", 1, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of topologies schedule pending", 0, rs.getUser("jerry").getTopologiesPending().size());
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
new file mode 100644
index 0000000..73a8c73
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
@@ -0,0 +1,305 @@
+/**
+ * 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 backtype.storm.scheduler.resource;
+
+import backtype.storm.Config;
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.INimbus;
+import backtype.storm.scheduler.IScheduler;
+import backtype.storm.scheduler.SchedulerAssignmentImpl;
+import backtype.storm.scheduler.SupervisorDetails;
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.scheduler.WorkerSlot;
+import backtype.storm.utils.Time;
+import backtype.storm.utils.Utils;
+import backtype.storm.validation.ConfigValidation;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+public class TestResourceAwareScheduler {
+
+    private static final int NUM_SUPS = 20;
+    private static final int NUM_WORKERS_PER_SUP = 4;
+    private final String TOPOLOGY_SUBMITTER = "jerry";
+
+    private static final Logger LOG = LoggerFactory.getLogger(TestResourceAwareScheduler.class);
+
+    @Test
+    public void TestReadInResourceAwareSchedulerUserPools() {
+
+        Map fromFile = Utils.findAndReadConfigFile("user-resource-pools.yaml", false);
+        LOG.info("fromFile: {}", fromFile);
+        ConfigValidation.validateFields(fromFile);
+    }
+
+    @Test
+    public void TestTopologySortedInCorrectOrder() {
+        INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
+        Map<String, Number> resourceMap = new HashMap<String, Number>();
+        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 100.0);
+        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1024.0);
+        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(20, 4, resourceMap);
+        Config config = new Config();
+        config.putAll(Utils.readDefaultConfig());
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, TOPOLOGY_SUBMITTER);
+
+        Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
+        resourceUserPool.put("jerry", new HashMap<String, Number>());
+        resourceUserPool.get("jerry").put("cpu", 1000);
+        resourceUserPool.get("jerry").put("memory", 8192.0);
+
+        resourceUserPool.put("bobby", new HashMap<String, Number>());
+        resourceUserPool.get("bobby").put("cpu", 10000.0);
+        resourceUserPool.get("bobby").put("memory", 32768);
+
+        resourceUserPool.put("derek", new HashMap<String, Number>());
+        resourceUserPool.get("derek").put("cpu", 5000.0);
+        resourceUserPool.get("derek").put("memory", 16384.0);
+
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
+
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30);
+
+
+        Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
+        topoMap.put(topo1.getId(), topo1);
+        topoMap.put(topo2.getId(), topo2);
+        topoMap.put(topo3.getId(), topo3);
+        topoMap.put(topo4.getId(), topo4);
+        topoMap.put(topo5.getId(), topo5);
+
+        Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), config);
+
+        Topologies topologies = new Topologies(topoMap);
+
+        ResourceAwareScheduler rs = new ResourceAwareScheduler();
+
+        rs.prepare(config);
+        rs.schedule(topologies, cluster);
+
+        Set<TopologyDetails> queue = rs.getUser("jerry").getTopologiesPending();
+        Assert.assertEquals("check size", queue.size(), 0);
+
+        queue = rs.getUser("jerry").getTopologiesRunning();
+
+        Iterator<TopologyDetails> itr = queue.iterator();
+
+        TopologyDetails topo = itr.next();
+        LOG.info("{} - {}", topo.getName(), queue);
+        Assert.assertEquals("check order", topo.getName(), "topo-4");
+
+        topo = itr.next();
+        LOG.info("{} - {}", topo.getName(), queue);
+        Assert.assertEquals("check order", topo.getName(), "topo-1");
+
+        topo = itr.next();
+        LOG.info("{} - {}", topo.getName(), queue);
+        Assert.assertEquals("check order", topo.getName(), "topo-5");
+
+        topo = itr.next();
+        LOG.info("{} - {}", topo.getName(), queue);
+        Assert.assertEquals("check order", topo.getName(), "topo-3");
+
+        topo = itr.next();
+        LOG.info("{} - {}", topo.getName(), queue);
+        Assert.assertEquals("check order", topo.getName(), "topo-2");
+
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 5, 15, 1, 1, Time.currentTimeSecs() - 30, 10);
+        topoMap.put(topo6.getId(), topo6);
+
+        topologies = new Topologies(topoMap);
+        rs = new ResourceAwareScheduler();
+        rs.prepare(config);
+        rs.schedule(topologies, cluster);
+
+        queue = rs.getUser("jerry").getTopologiesRunning();
+        itr = queue.iterator();
+
+        topo = itr.next();
+        Assert.assertEquals("check order", topo.getName(), "topo-6");
+
+        topo = itr.next();
+        Assert.assertEquals("check order", topo.getName(), "topo-4");
+
+        topo = itr.next();
+        Assert.assertEquals("check order", topo.getName(), "topo-1");
+
+        topo = itr.next();
+        Assert.assertEquals("check order", topo.getName(), "topo-5");
+
+        topo = itr.next();
+        Assert.assertEquals("check order", topo.getName(), "topo-3");
+
+        topo = itr.next();
+        Assert.assertEquals("check order", topo.getName(), "topo-2");
+
+        queue = rs.getUser("jerry").getTopologiesPending();
+        Assert.assertEquals("check size", queue.size(), 0);
+    }
+
+    @Test
+    public void TestMultipleUsers() {
+        INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
+        Map<String, Number> resourceMap = new HashMap<String, Number>();
+        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 1000.0);
+        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1024.0 * 10);
+        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(20, 4, resourceMap);
+        Config config = new Config();
+        config.putAll(Utils.readDefaultConfig());
+        Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
+        resourceUserPool.put("jerry", new HashMap<String, Number>());
+        resourceUserPool.get("jerry").put("cpu", 1000);
+        resourceUserPool.get("jerry").put("memory", 8192.0);
+
+        resourceUserPool.put("bobby", new HashMap<String, Number>());
+        resourceUserPool.get("bobby").put("cpu", 10000.0);
+        resourceUserPool.get("bobby").put("memory", 32768);
+
+        resourceUserPool.put("derek", new HashMap<String, Number>());
+        resourceUserPool.get("derek").put("cpu", 5000.0);
+        resourceUserPool.get("derek").put("memory", 16384.0);
+
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
+        Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), config);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
+
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
+
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo7 = TestUtilsForResourceAwareScheduler.getTopology("topo-7", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
+        TopologyDetails topo8 = TestUtilsForResourceAwareScheduler.getTopology("topo-8", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30);
+        TopologyDetails topo9 = TestUtilsForResourceAwareScheduler.getTopology("topo-9", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
+        TopologyDetails topo10 = TestUtilsForResourceAwareScheduler.getTopology("topo-10", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
+
+        TopologyDetails topo11 = TestUtilsForResourceAwareScheduler.getTopology("topo-11", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo12 = TestUtilsForResourceAwareScheduler.getTopology("topo-12", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
+        TopologyDetails topo13 = TestUtilsForResourceAwareScheduler.getTopology("topo-13", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30);
+        TopologyDetails topo14 = TestUtilsForResourceAwareScheduler.getTopology("topo-14", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
+        TopologyDetails topo15 = TestUtilsForResourceAwareScheduler.getTopology("topo-15", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30);
+
+        Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
+        topoMap.put(topo1.getId(), topo1);
+        topoMap.put(topo2.getId(), topo2);
+        topoMap.put(topo3.getId(), topo3);
+        topoMap.put(topo4.getId(), topo4);
+        topoMap.put(topo5.getId(), topo5);
+        topoMap.put(topo6.getId(), topo6);
+        topoMap.put(topo7.getId(), topo7);
+        topoMap.put(topo8.getId(), topo8);
+        topoMap.put(topo9.getId(), topo9);
+        topoMap.put(topo10.getId(), topo10);
+        topoMap.put(topo11.getId(), topo11);
+        topoMap.put(topo12.getId(), topo12);
+        topoMap.put(topo13.getId(), topo13);
+        topoMap.put(topo14.getId(), topo14);
+        topoMap.put(topo15.getId(), topo15);
+
+        Topologies topologies = new Topologies(topoMap);
+
+        ResourceAwareScheduler rs = new ResourceAwareScheduler();
+
+        rs.prepare(config);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : topoMap.values()) {
+            Assert.assertEquals(cluster.getStatusMap().get(topo.getId()), "Fully Scheduled");
+        }
+
+        for (User user : rs.getUserMap().values()) {
+            Assert.assertEquals(user.getTopologiesPending().size(), 0);
+            Assert.assertEquals(user.getTopologiesRunning().size(), 5);
+        }
+    }
+
+    @Test
+    public void testHandlingClusterSubscription() {
+        INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
+        Map<String, Number> resourceMap = new HashMap<String, Number>();
+        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 200.0);
+        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1024.0 * 10);
+        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(1, 4, resourceMap);
+        Config config = new Config();
+        config.putAll(Utils.readDefaultConfig());
+        Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
+        resourceUserPool.put("jerry", new HashMap<String, Number>());
+        resourceUserPool.get("jerry").put("cpu", 1000);
+        resourceUserPool.get("jerry").put("memory", 8192.0);
+
+        resourceUserPool.put("bobby", new HashMap<String, Number>());
+        resourceUserPool.get("bobby").put("cpu", 10000.0);
+        resourceUserPool.get("bobby").put("memory", 32768);
+
+        resourceUserPool.put("derek", new HashMap<String, Number>());
+        resourceUserPool.get("derek").put("cpu", 5000.0);
+        resourceUserPool.get("derek").put("memory", 16384.0);
+
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
+        Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), config);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
+
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
+
+        Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
+        topoMap.put(topo1.getId(), topo1);
+        topoMap.put(topo2.getId(), topo2);
+
+        Topologies topologies = new Topologies(topoMap);
+
+        ResourceAwareScheduler rs = new ResourceAwareScheduler();
+
+        rs.prepare(config);
+        rs.schedule(topologies, cluster);
+
+        int fullyScheduled = 0;
+        for (TopologyDetails topo : topoMap.values()) {
+            if(cluster.getStatusMap().get(topo.getId()).equals("Fully Scheduled")) {
+                fullyScheduled++;
+            }
+        }
+        Assert.assertEquals("# of Fully scheduled", 1, fullyScheduled);
+        Assert.assertEquals("# of topologies schedule attempted", 1, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of topologies running", 1, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of topologies schedule pending", 0, rs.getUser("jerry").getTopologiesPending().size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUser.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUser.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUser.java
new file mode 100644
index 0000000..e9bf039
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUser.java
@@ -0,0 +1,111 @@
+/**
+ * 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 backtype.storm.scheduler.resource;
+
+import backtype.storm.Config;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.utils.Time;
+import backtype.storm.utils.Utils;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+public class TestUser {
+
+    private static final Logger LOG = LoggerFactory.getLogger(TestUser.class);
+
+    @Test
+    public void testAddTopologyToPendingQueue() {
+
+        Config config = new Config();
+        config.putAll(Utils.readDefaultConfig());
+
+        List<TopologyDetails> topos = TestUtilsForResourceAwareScheduler.getListOfTopologies(config);
+        User user1 = new User("user1");
+
+        for (TopologyDetails topo : topos) {
+            user1.addTopologyToPendingQueue(topo);
+        }
+
+        Assert.assertTrue(user1.getTopologiesPending().size() == topos.size());
+
+        List<String> correctOrder = TestUtilsForResourceAwareScheduler.getListOfTopologiesCorrectOrder();
+        Iterator<String> itr = correctOrder.iterator();
+        for (TopologyDetails topo : user1.getTopologiesPending()) {
+            Assert.assertEquals("check order", topo.getName(), itr.next());
+        }
+    }
+
+    @Test
+    public void testMoveTopoFromPendingToRunning() {
+
+        Config config = new Config();
+        config.putAll(Utils.readDefaultConfig());
+
+        List<TopologyDetails> topos = TestUtilsForResourceAwareScheduler.getListOfTopologies(config);
+        User user1 = new User("user1");
+
+        for (TopologyDetails topo : topos) {
+            user1.addTopologyToPendingQueue(topo);
+        }
+
+        int counter = 1;
+        for (TopologyDetails topo : topos) {
+            user1.moveTopoFromPendingToRunning(topo);
+            Assert.assertEquals("check correct size", (topos.size() - counter), user1.getTopologiesPending().size());
+            Assert.assertEquals("check correct size", counter, user1.getTopologiesRunning().size());
+            counter++;
+        }
+    }
+
+    @Test
+    public void testResourcePoolUtilization() {
+
+        Double cpuGuarantee = 400.0;
+        Double memoryGuarantee = 1000.0;
+        Map<String, Double> resourceGuaranteeMap = new HashMap<String, Double>();
+        resourceGuaranteeMap.put("cpu", cpuGuarantee);
+        resourceGuaranteeMap.put("memory", memoryGuarantee);
+
+        Config config = new Config();
+        config.putAll(Utils.readDefaultConfig());
+        config.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 100);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 200);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 200);
+
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 1, 2, 1, Time.currentTimeSecs() - 24, 9);
+
+        User user1 = new User("user1", resourceGuaranteeMap);
+
+        user1.addTopologyToRunningQueue(topo1);
+
+        Assert.assertEquals("check cpu resource guarantee", cpuGuarantee, user1.getCPUResourceGuaranteed(), 0.001);
+        Assert.assertEquals("check memory resource guarantee", memoryGuarantee, user1.getMemoryResourceGuaranteed(), 0.001);
+
+        Assert.assertEquals("check cpu resource pool utilization", ((100.0 * 3.0) / cpuGuarantee), user1.getCPUResourcePoolUtilization().doubleValue(), 0.001);
+        Assert.assertEquals("check memory resource pool utilization", ((200.0 + 200.0) * 3.0) / memoryGuarantee, user1.getMemoryResourcePoolUtilization().doubleValue(), 0.001);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
new file mode 100644
index 0000000..d4177c3
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
@@ -0,0 +1,265 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * 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 backtype.storm.scheduler.resource;
+
+import backtype.storm.Config;
+import backtype.storm.generated.Bolt;
+import backtype.storm.generated.SpoutSpec;
+import backtype.storm.generated.StormTopology;
+import backtype.storm.scheduler.ExecutorDetails;
+import backtype.storm.scheduler.INimbus;
+import backtype.storm.scheduler.IScheduler;
+import backtype.storm.scheduler.SupervisorDetails;
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.scheduler.WorkerSlot;
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.BoltDeclarer;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.SpoutDeclarer;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.topology.base.BaseRichBolt;
+import backtype.storm.topology.base.BaseRichSpout;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+import backtype.storm.utils.Time;
+import backtype.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+public class TestUtilsForResourceAwareScheduler {
+    private static final Logger LOG = LoggerFactory.getLogger(TestUtilsForResourceAwareScheduler.class);
+
+    public static List<TopologyDetails> getListOfTopologies(Config config) {
+
+        List<TopologyDetails> topos = new LinkedList<TopologyDetails>();
+
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 0));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-7", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 0));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-8", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 15));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-9", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 8));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-10", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 9));
+        return topos;
+    }
+
+    public static List<String> getListOfTopologiesCorrectOrder() {
+        List<String> topos = new LinkedList<String>();
+        topos.add("topo-7");
+        topos.add("topo-6");
+        topos.add("topo-9");
+        topos.add("topo-10");
+        topos.add("topo-8");
+        topos.add("topo-4");
+        topos.add("topo-1");
+        topos.add("topo-5");
+        topos.add("topo-3");
+        topos.add("topo-2");
+        return topos;
+    }
+
+    public static Map<String, SupervisorDetails> genSupervisors(int numSup, int numPorts, Map resourceMap) {
+        Map<String, SupervisorDetails> retList = new HashMap<String, SupervisorDetails>();
+        for(int i=0; i<numSup; i++) {
+            List<Number> ports = new LinkedList<Number>();
+            for(int j = 0; j<numPorts; j++) {
+                ports.add(j);
+            }
+            SupervisorDetails sup = new SupervisorDetails("sup-"+i, "host-"+i, null, ports, resourceMap);
+            retList.put(sup.getId(), sup);
+        }
+        return retList;
+    }
+
+    public static Map<ExecutorDetails, String> genExecsAndComps(StormTopology topology, int spoutParallelism, int boltParallelism) {
+        Map<ExecutorDetails, String> retMap = new HashMap<ExecutorDetails, String> ();
+        int startTask=0;
+        int endTask=1;
+        for(Map.Entry<String, SpoutSpec> entry : topology.get_spouts().entrySet()) {
+            for(int i=0; i<spoutParallelism; i++) {
+                retMap.put(new ExecutorDetails(startTask, endTask), entry.getKey());
+                startTask++;
+                endTask++;
+            }
+        }
+
+        for(Map.Entry<String, Bolt> entry : topology.get_bolts().entrySet()) {
+            for(int i=0; i<boltParallelism; i++) {
+                retMap.put(new ExecutorDetails(startTask, endTask), entry.getKey());
+                startTask++;
+                endTask++;
+            }
+        }
+        return retMap;
+    }
+
+    public static TopologyDetails getTopology(String name, Map config, int numSpout, int numBolt,
+                                       int spoutParallelism, int boltParallelism, int launchTime, int priority) {
+
+        Config conf = new Config();
+        conf.putAll(config);
+        conf.put(Config.TOPOLOGY_PRIORITY, priority);
+        conf.put(Config.TOPOLOGY_NAME, name);
+        StormTopology topology = buildTopology(numSpout,numBolt, spoutParallelism, boltParallelism);
+        TopologyDetails topo = new TopologyDetails(name + "-" + launchTime, conf, topology,
+                0,
+                genExecsAndComps(topology, spoutParallelism, boltParallelism), launchTime);
+        return topo;
+    }
+
+    public static StormTopology buildTopology(int numSpout, int numBolt,
+                                              int spoutParallelism, int boltParallelism) {
+        LOG.debug("buildTopology with -> numSpout: " + numSpout + " spoutParallelism: "
+                + spoutParallelism + " numBolt: "
+                + numBolt + " boltParallelism: " + boltParallelism);
+        TopologyBuilder builder = new TopologyBuilder();
+
+        for (int i = 0; i < numSpout; i++) {
+            SpoutDeclarer s1 = builder.setSpout("spout-" + i, new TestSpout(),
+                    spoutParallelism);
+        }
+        int j = 0;
+        for (int i = 0; i < numBolt; i++) {
+            if (j >= numSpout) {
+                j = 0;
+            }
+            BoltDeclarer b1 = builder.setBolt("bolt-" + i, new TestBolt(),
+                    boltParallelism).shuffleGrouping("spout-" + j);
+        }
+
+        return builder.createTopology();
+    }
+
+    public static class TestSpout extends BaseRichSpout {
+        boolean _isDistributed;
+        SpoutOutputCollector _collector;
+
+        public TestSpout() {
+            this(true);
+        }
+
+        public TestSpout(boolean isDistributed) {
+            _isDistributed = isDistributed;
+        }
+
+        public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+            _collector = collector;
+        }
+
+        public void close() {
+
+        }
+
+        public void nextTuple() {
+            Utils.sleep(100);
+            final String[] words = new String[]{"nathan", "mike", "jackson", "golda", "bertels"};
+            final Random rand = new Random();
+            final String word = words[rand.nextInt(words.length)];
+            _collector.emit(new Values(word));
+        }
+
+        public void ack(Object msgId) {
+
+        }
+
+        public void fail(Object msgId) {
+
+        }
+
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("word"));
+        }
+
+        @Override
+        public Map<String, Object> getComponentConfiguration() {
+            if (!_isDistributed) {
+                Map<String, Object> ret = new HashMap<String, Object>();
+                ret.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, 1);
+                return ret;
+            } else {
+                return null;
+            }
+        }
+    }
+
+    public static class TestBolt extends BaseRichBolt {
+        OutputCollector _collector;
+
+        @Override
+        public void prepare(Map conf, TopologyContext context,
+                            OutputCollector collector) {
+            _collector = collector;
+        }
+
+        @Override
+        public void execute(Tuple tuple) {
+            _collector.emit(tuple, new Values(tuple.getString(0) + "!!!"));
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("word"));
+        }
+    }
+
+    public static class INimbusTest implements INimbus {
+        @Override
+        public void prepare(Map stormConf, String schedulerLocalDir) {
+
+        }
+
+        @Override
+        public Collection<WorkerSlot> allSlotsAvailableForScheduling(Collection<SupervisorDetails> existingSupervisors, Topologies topologies, Set<String> topologiesMissingAssignments) {
+            return null;
+        }
+
+        @Override
+        public void assignSlots(Topologies topologies, Map<String, Collection<WorkerSlot>> newSlotsByTopologyId) {
+
+        }
+
+        @Override
+        public String getHostName(Map<String, SupervisorDetails> existingSupervisors, String nodeId) {
+            if(existingSupervisors.containsKey(nodeId)) {
+                return existingSupervisors.get(nodeId).getHost();
+            }
+            return null;
+        }
+
+        @Override
+        public IScheduler getForcedScheduler() {
+            return null;
+        }
+    }
+}


[12/23] storm git commit: adding documentation

Posted by da...@apache.org.
adding documentation


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

Branch: refs/heads/master
Commit: f5bccd05e434c2f0e6007c8c786ef33fa14169e3
Parents: 88ad3c3
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Sat Dec 5 16:50:41 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Sat Dec 5 16:50:41 2015 -0600

----------------------------------------------------------------------
 .../Resource_Aware_Scheduler_overview.md        | 222 +++++++++++++++++++
 ...ware_scheduler_default_eviction_strategy.png | Bin 0 -> 188099 bytes
 2 files changed, 222 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/f5bccd05/docs/documentation/Resource_Aware_Scheduler_overview.md
----------------------------------------------------------------------
diff --git a/docs/documentation/Resource_Aware_Scheduler_overview.md b/docs/documentation/Resource_Aware_Scheduler_overview.md
new file mode 100644
index 0000000..0ca8d09
--- /dev/null
+++ b/docs/documentation/Resource_Aware_Scheduler_overview.md
@@ -0,0 +1,222 @@
+# Introduction
+
+The purpose of this document is to provide a description of the Resource Aware Scheduler for the Storm distributed real-time computation system.  This document will provide you with both a high level description of the resource aware scheduler in Storm
+
+## Using Resource Aware Scheduler
+
+The user can switch to using the Resource Aware Scheduler by setting the following in *conf/storm.yaml*
+
+    storm.scheduler: “backtype.storm.scheduler.resource.ResourceAwareScheduler”
+
+
+## API Overview
+
+For a Storm Topology, the user can now specify the amount of resources a topology component (i.e. Spout or Bolt) is required to run a single instance of the component.  The user can specify the resource requirement for a topology component by using the following API calls.
+
+### Setting Memory Requirement
+
+API to set component memory requirement:
+
+    public T setMemoryLoad(Number onHeap, Number offHeap)
+
+Parameters:
+* Number onHeap – The amount of on heap memory an instance of this component will consume in megabytes
+* Number OffHeap – The amount of off heap memory an instance of this component will consume in megabytes
+
+The user also have to option to just specify the on heap memory requirement if the component does not have an off heap memory need.
+
+    public T setMemoryLoad(Number onHeap)
+
+Parameters:
+* Number onHeap – The amount of on heap memory an instance of this component will consume
+
+If no value is provided for offHeap, 0.0 will be used. If no value is provided for onHeap, or if the API is never called for a component, the default value will be used.
+
+Example of Usage:
+
+    SpoutDeclarer s1 = builder.setSpout("word", new TestWordSpout(), 10);
+    s1.setMemoryLoad(1024.0, 512.0);
+    builder.setBolt("exclaim1", new ExclamationBolt(), 3)
+                .shuffleGrouping("word").setMemoryLoad(512.0);
+
+The entire memory requested for this topology is 16.5 GB. That is from 10 spouts with 1GB on heap memory and 0.5 GB off heap memory each and 3 bolts with 0.5 GB on heap memory each.
+
+### Setting CPU Requirement
+
+
+API to set component CPU requirement:
+
+    public T setCPULoad(Double amount)
+
+Parameters:
+* Number amount – The amount of on CPU an instance of this component will consume.
+
+Currently, the amount of CPU resources a component requires or is available on a node is represented by a point system. CPU usage is a difficult concept to define. Different CPU architectures perform differently depending on the task at hand. They are so complex that expressing all of that in a single precise portable number is impossible. Instead we take a convention over configuration approach and are primarily concerned with rough level of CPU usage while still providing the possibility to specify amounts more fine grained.
+
+By convention a CPU core typically will get 100 points. If you feel that your processors are more or less powerful you can adjust this accordingly. Heavy tasks that are CPU bound will get 100 points, as they can consume an entire core. Medium tasks should get 50, light tasks 25, and tiny tasks 10. In some cases you have a task that spawns other threads to help with processing. These tasks may need to go above 100 points to express the amount of CPU they are using. If these conventions are followed the common case for a single threaded task the reported Capacity * 100 should be the number of CPU points that the task needs.
+
+Example of Usage:
+
+    SpoutDeclarer s1 = builder.setSpout("word", new TestWordSpout(), 10);
+    s1.setCPULoad(15.0);
+    builder.setBolt("exclaim1", new ExclamationBolt(), 3)
+                .shuffleGrouping("word").setCPULoad(10.0);
+
+###	Limiting the Heap Size per Worker (JVM) Process
+
+
+    public void setTopologyWorkerMaxHeapSize(Number size)
+
+Parameters:
+* Number size – The memory limit a worker process will be allocated in megabytes
+
+The user can limit the amount of memory resources the resource aware scheduler that is allocated to a single worker on a per topology basis by using the above API.  This API is in place so that the users can spread executors to multiple workers.  However, spreading workers to multiple workers may increase the communication latency since executors will not be able to use Disruptor Queue for intra-process communication.
+
+Example of Usage:
+
+    Config conf = new Config();
+    conf.setTopologyWorkerMaxHeapSize(512.0);
+
+### Setting Available Resources on Node
+
+A storm administrator can specify node resource availability by modifying the *conf/storm.yaml* file located in the storm home directory of that node.
+
+A storm administrator can specify how much available memory a node has in megabytes adding the following to *storm.yaml*
+
+    supervisor.memory.capacity.mb: [amount<Double>]
+
+A storm administrator can also specify how much available CPU resources a node has available adding the following to *storm.yaml*
+
+    supervisor.cpu.capacity: [amount<Double>]
+
+
+Note: that the amount the user can specify for the available CPU is represented using a point system like discussed earlier.
+
+Example of Usage:
+
+    supervisor.memory.capacity.mb: 20480.0
+    supervisor.cpu.capacity: 100.0
+
+
+2.5.	Other Configurations
+
+The user can set some default configurations for the Resource Aware Scheduler in *conf/storm.yaml*:
+
+    //default value if on heap memory requirement is not specified for a component 
+    topology.component.resources.onheap.memory.mb: 128.0
+
+    //default value if off heap memory requirement is not specified for a component 
+    topology.component.resources.offheap.memory.mb: 0.0
+
+    //default value if CPU requirement is not specified for a component 
+    topology.component.cpu.pcore.percent: 10.0
+
+    //default value for the max heap size for a worker  
+    topology.worker.max.heap.size.mb: 768.0
+
+# Topology Priorities and Per User Resource 
+
+The next step for the Resource Aware Scheduler or RAS is to enable it to have multitenant capabilities since many Storm users typically share a Storm cluster.  Resource Aware Scheduler needs to be able to allocate resources on a per user basis.  Each user can be guaranteed a certain amount of resources to run his or her topologies and the Resource Aware Scheduler should meet those guarantees when possible.  When the Storm cluster has extra free resources, Resource Aware Scheduler needs to be able allocate additional resources to user in a fair manner. The importance of topologies can also vary.  Topologies can be used for actual production or just experimentation, thus Resource Aware Scheduler should take into account the importance of a topology when determining the order in which to schedule topologies or when to evict topologies
+
+## Setup
+
+The resource guarantees of a user can be specified *conf/user-resource-pools.yaml*.  Specify the resource guarantees of a user in the following format:
+
+    resource.aware.scheduler.user.pools:
+	[UserId]
+		cpu: [Amount of Guarantee CPU Resources]
+		memory: [Amount of Guarantee Memory Resources]
+
+An example of what *user-resource-pools.yaml* can look like:
+
+    resource.aware.scheduler.user.pools:
+        jerry:
+            cpu: 1000
+            memory: 8192.0
+        derek:
+            cpu: 10000.0
+            memory: 32768
+        bobby:
+            cpu: 5000.0
+            memory: 16384.0
+
+Please note that the specified amount of Guaranteed CPU and Memory can be either a integer or double
+
+## API Overview
+### Specifying topology priority
+The range of topology priorities can range form 0-30.  The topologies priorities will be partitioned into several priority levels that may contain a range of priorities. 
+For example we can create a priority level mapping:
+
+    PRODUCTION => 0 – 9
+    STAGING => 10 – 19
+    DEV => 20 – 29
+
+Thus, each priority level contains 10 sub priorities. Users can set the priority level of a topology by using the following API
+
+    conf.setTopologyPriority(int priority)
+Parameters:
+* priority – an integer representing the priority of the topology
+
+### Specifying Scheduling Strategy:
+
+A user can specify on a per topology basis what scheduling strategy to use.  Users can implement the IStrategy interface and define a new strategies to schedule specific topologies.  This pluggable interface was created since we realize different topologies might have different scheduling needs.  A user can set the topology strategy within the topology definition by using the API:
+
+    public void setTopologyStrategy(Class<? extends IStrategy> clazz)
+    
+Parameters:
+* clazz – The strategy class that implements the IStrategy interface
+
+Example Usage:
+    conf.setTopologyStrategy(backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class);
+
+A default scheduling is provided.  The DefaultResourceAwareStrategy is implemented based off the scheduling algorithm in the original paper describing resource aware scheduling in Storm:
+
+http://web.engr.illinois.edu/~bpeng/files/r-storm.pdf
+
+### Specifying Topology Prioritization Strategy
+
+The order of scheduling is a pluggable interface in which a user could define a strategy that prioritizes topologies.  For a user to define his or her own prioritization strategy, he or she needs to implement the ISchedulingPriorityStrategy interface.  A user can set the scheduling priority strategy by setting the *Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY* to point to the class that implements the strategy. For instance:
+
+    resource.aware.scheduler.priority.strategy: "backtype.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy"
+    
+    A default strategy will be provided.  The following explains how the default scheduling priority strategy works.
+
+**DefaultSchedulingPriorityStrategy**
+
+The order of scheduling should be based on how far is a user’s current resource allocation to his or her guaranteed allocation.  We should prioritize the users who are the furthest away from their resource guarantee. The difficulty of this problem is that a user may have multiple resource guarantees and another user can have another set resource guarantees, so how can we compare them in a fair manner?  Lets use the average percentage of resource guarantees satisfied as a method of comparison.
+
+For example:
+
+|User|Resource Guarantee|Resource Allocated|
+|----|------------------|------------------|
+|A|<10 CPU, 50GB>|<2 CPU, 40 GB>|
+|B|< 20 CPU, 25GB>|<15 CPU, 10 GB>|
+
+User A’s average percentage satisfied of resource guarantee: 
+
+(2/10+40/50)/2  = 0.5
+
+User B’s average percentage satisfied of resource guarantee: 
+
+(15/20+10/25)/2  = 0.575
+
+Thus, in this example User A has a smaller average percentage of his or her resource guarantee satisfied than User B.  Thus, User A should get priority to be allocated more resource, i.e., schedule a topology submitted by User A.
+
+When scheduling, RAS sort users by the average percentage satisfied of resource guarantee and schedule topologies from users based on that ordering starting from the users with the lowest average percentage satisfied of resource guarantee.  When a user’s resource guarantee is completely satisfied, the user’s average percentage satisfied of resource guarantee will be greater than or equal to 1.
+
+### Specifying Eviction Strategy
+The strategy for evicting topologies is also a pluggable interface in which the user can implement his or her own topology eviction strategy.  For a user to implement his or her own eviction strategy, he or she needs to implement the IEvictionStrategy Interface and set *Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY* to point to the implemented strategy class. For instance:
+
+    resource.aware.scheduler.eviction.strategy: "backtype.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy"
+
+A default eviction strategy is provided.  The following explains how the default topology eviction strategy works
+
+**DefaultEvictionStrategy**
+
+If the cluster is full, we need a mechanism to evict topologies so that user resource guarantees can be met and resources additional resource guarantees can be shared fairly among users
+
+To determine if topology eviction should occur we should take into account the priority of the topology that we are trying to schedule and whether the resource guarantees for the owner of the topology have been met.  
+
+We should never evict a topology from a user that does not have his or her resource guarantees satisfied.  The following flow chart should describe the logic for the eviction process.
+
+![Viewing metrics with VisualVM](images/resource_aware_scheduler_default_eviction_strategy.png)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/f5bccd05/docs/images/resource_aware_scheduler_default_eviction_strategy.png
----------------------------------------------------------------------
diff --git a/docs/images/resource_aware_scheduler_default_eviction_strategy.png b/docs/images/resource_aware_scheduler_default_eviction_strategy.png
new file mode 100644
index 0000000..3c07e1f
Binary files /dev/null and b/docs/images/resource_aware_scheduler_default_eviction_strategy.png differ


[19/23] storm git commit: edits based on review comments

Posted by da...@apache.org.
edits based on review comments


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

Branch: refs/heads/master
Commit: 0f80d0678c243a7bd1b6aa169272741ef7e2950a
Parents: 068a8c2
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Fri Dec 18 01:09:15 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Fri Dec 18 01:21:41 2015 -0600

----------------------------------------------------------------------
 .../jvm/backtype/storm/scheduler/Cluster.java   |   2 +-
 .../resource/ResourceAwareScheduler.java        |  88 +++----
 .../storm/scheduler/resource/ResourceUtils.java |  11 +-
 .../scheduler/resource/SchedulingResult.java    |   2 +-
 .../backtype/storm/scheduler/resource/User.java |   3 +-
 .../eviction/DefaultEvictionStrategy.java       |  10 +-
 .../DefaultSchedulingPriorityStrategy.java      |   4 -
 .../strategies/scheduling/IStrategy.java        |   4 -
 .../jvm/backtype/storm/TestConfigValidate.java  |  13 +-
 .../resource/TestResourceAwareScheduler.java    | 229 ++++++++++++-------
 .../TestUtilsForResourceAwareScheduler.java     |  26 +--
 11 files changed, 208 insertions(+), 184 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/0f80d067/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
index 53fdaa4..e3297c8 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
@@ -458,7 +458,7 @@ public class Cluster {
      * set assignments for cluster
      */
     public void setAssignments(Map<String, SchedulerAssignment> newAssignments) {
-        this.assignments = new HashMap<String, SchedulerAssignmentImpl>();
+        this.assignments = new HashMap<String, SchedulerAssignmentImpl>(newAssignments.size());
         for (Map.Entry<String, SchedulerAssignment> entry : newAssignments.entrySet()) {
             this.assignments.put(entry.getKey(), new SchedulerAssignmentImpl(entry.getValue().getTopologyId(), entry.getValue().getExecutorToSlot()));
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/0f80d067/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
index 0558e12..116f1b5 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
@@ -99,9 +99,9 @@ public class ResourceAwareScheduler implements IScheduler {
             if (schedulingPrioritystrategy == null) {
                 try {
                     schedulingPrioritystrategy = (ISchedulingPriorityStrategy) Utils.newInstance((String) this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY));
-                } catch (RuntimeException e) {
-                    LOG.error("failed to create instance of priority strategy: {} with error: {}! No topologies will be scheduled.",
-                            this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY), e.getMessage());
+                } catch (RuntimeException ex) {
+                    LOG.error(String.format("failed to create instance of priority strategy: %s with error: %s! No topologies will be scheduled.",
+                                    this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY), ex.getMessage()), ex);
                     break;
                 }
             }
@@ -111,9 +111,9 @@ public class ResourceAwareScheduler implements IScheduler {
                 schedulingPrioritystrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
                 //Call scheduling priority strategy
                 td = schedulingPrioritystrategy.getNextTopologyToSchedule();
-            } catch (Exception e) {
-                LOG.error("Exception thrown when running priority strategy {}. No topologies will be scheduled! Error: {} StackTrace: {}"
-                        , schedulingPrioritystrategy.getClass().getName(), e.getMessage(), Arrays.toString(e.getStackTrace()));
+            } catch (Exception ex) {
+                LOG.error(String.format("Exception thrown when running priority strategy %s. No topologies will be scheduled! Error: %s"
+                        , schedulingPrioritystrategy.getClass().getName(), ex.getMessage()), ex.getStackTrace());
                 break;
             }
             if (td == null) {
@@ -135,9 +135,7 @@ public class ResourceAwareScheduler implements IScheduler {
             } catch (RuntimeException e) {
                 LOG.error("failed to create instance of IStrategy: {} with error: {}! Topology {} will not be scheduled.",
                         td.getName(), td.getConf().get(Config.TOPOLOGY_SCHEDULER_STRATEGY), e.getMessage());
-                this.restoreCheckpointSchedulingState(schedulingState);
-                //since state is restored need the update User topologySubmitter to the new User object in userMap
-                topologySubmitter = this.userMap.get(td.getTopologySubmitter());
+                topologySubmitter = this.cleanup(schedulingState, td);
                 topologySubmitter.moveTopoFromPendingToInvalid(td);
                 this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling - failed to create instance of topology strategy "
                         + td.getConf().get(Config.TOPOLOGY_SCHEDULER_STRATEGY) + ". Please check logs for details");
@@ -150,12 +148,10 @@ public class ResourceAwareScheduler implements IScheduler {
                     //Need to re prepare scheduling strategy with cluster and topologies in case scheduling state was restored
                     rasStrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
                     result = rasStrategy.schedule(td);
-                } catch (Exception e) {
-                    LOG.error("Exception thrown when running strategy {} to schedule topology {}. Topology will not be scheduled! Error: {} StackTrace: {}"
-                            , rasStrategy.getClass().getName(), td.getName(), e.getMessage(), Arrays.toString(e.getStackTrace()));
-                    this.restoreCheckpointSchedulingState(schedulingState);
-                    //since state is restored need the update User topologySubmitter to the new User object in userMap
-                    topologySubmitter = this.userMap.get(td.getTopologySubmitter());
+                } catch (Exception ex) {
+                    LOG.error(String.format("Exception thrown when running strategy %s to schedule topology %s. Topology will not be scheduled!"
+                            , rasStrategy.getClass().getName(), td.getName()), ex);
+                    topologySubmitter = this.cleanup(schedulingState, td);
                     topologySubmitter.moveTopoFromPendingToInvalid(td);
                     this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling - Exception thrown when running strategy {}"
                             + rasStrategy.getClass().getName() + ". Please check logs for details");
@@ -168,19 +164,13 @@ public class ResourceAwareScheduler implements IScheduler {
                                 topologySubmitter.moveTopoFromPendingToRunning(td);
                                 this.cluster.setStatus(td.getId(), "Running - " + result.getMessage());
                             } else {
-                                this.restoreCheckpointSchedulingState(schedulingState);
-                                //since state is restored need the update User topologySubmitter to the new User object in userMap
-                                topologySubmitter = this.userMap.get(td.getTopologySubmitter());
+                                topologySubmitter = this.cleanup(schedulingState, td);
                                 topologySubmitter.moveTopoFromPendingToAttempted(td);
                                 this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling - Unable to assign executors to nodes. Please check logs for details");
                             }
                         } catch (IllegalStateException ex) {
-                            LOG.error(ex.toString());
-                            LOG.error("Unsuccessful in scheduling - IllegalStateException thrown when attempting to assign executors to nodes. Error: {} StackTrace: {}",
-                                    ex.getClass().getName(), Arrays.toString(ex.getStackTrace()));
-                            this.restoreCheckpointSchedulingState(schedulingState);
-                            //since state is restored need the update User topologySubmitter to the new User object in userMap
-                            topologySubmitter = this.userMap.get(td.getTopologySubmitter());
+                            LOG.error("Unsuccessful in scheduling - IllegalStateException thrown when attempting to assign executors to nodes.", ex);
+                            topologySubmitter = this.cleanup(schedulingState, td);
                             topologySubmitter.moveTopoFromPendingToAttempted(td);
                             this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling - IllegalStateException thrown when attempting to assign executors to nodes. Please check log for details.");
                         }
@@ -202,44 +192,34 @@ public class ResourceAwareScheduler implements IScheduler {
                                 //need to re prepare since scheduling state might have been restored
                                 evictionStrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
                                 madeSpace = evictionStrategy.makeSpaceForTopo(td);
-                            } catch (Exception e) {
-                                LOG.error("Exception thrown when running eviction strategy {} to schedule topology {}. No evictions will be done! Error: {} StackTrace: {}"
-                                        , evictionStrategy.getClass().getName(), td.getName(), e.getClass().getName(), Arrays.toString(e.getStackTrace()));
-                                this.restoreCheckpointSchedulingState(schedulingState);
-                                //since state is restored need the update User topologySubmitter to the new User object in userMap
-                                topologySubmitter = this.userMap.get(td.getTopologySubmitter());
+                            } catch (Exception ex) {
+                                LOG.error(String.format("Exception thrown when running eviction strategy %s to schedule topology %s. No evictions will be done! Error: %s"
+                                        , evictionStrategy.getClass().getName(), td.getName(), ex.getClass().getName()), ex);
+                                topologySubmitter = this.cleanup(schedulingState, td);
                                 topologySubmitter.moveTopoFromPendingToAttempted(td);
                                 break;
                             }
                             if (!madeSpace) {
                                 LOG.debug("Could not make space for topo {} will move to attempted", td);
-                                this.restoreCheckpointSchedulingState(schedulingState);
-                                //since state is restored need the update User topologySubmitter to the new User object in userMap
-                                topologySubmitter = this.userMap.get(td.getTopologySubmitter());
+                                topologySubmitter = this.cleanup(schedulingState, td);
                                 topologySubmitter.moveTopoFromPendingToAttempted(td);
                                 this.cluster.setStatus(td.getId(), "Not enough resources to schedule - " + result.getErrorMessage());
                                 break;
                             }
                             continue;
                         } else if (result.getStatus() == SchedulingStatus.FAIL_INVALID_TOPOLOGY) {
-                            this.restoreCheckpointSchedulingState(schedulingState);
-                            //since state is restored need the update User topologySubmitter to the new User object in userMap
-                            topologySubmitter = this.userMap.get(td.getTopologySubmitter());
+                            topologySubmitter = this.cleanup(schedulingState, td);
                             topologySubmitter.moveTopoFromPendingToInvalid(td, this.cluster);
                             break;
                         } else {
-                            this.restoreCheckpointSchedulingState(schedulingState);
-                            //since state is restored need the update User topologySubmitter to the new User object in userMap
-                            topologySubmitter = this.userMap.get(td.getTopologySubmitter());
+                            topologySubmitter = this.cleanup(schedulingState, td);
                             topologySubmitter.moveTopoFromPendingToAttempted(td, this.cluster);
                             break;
                         }
                     }
                 } else {
                     LOG.warn("Scheduling results returned from topology {} is not vaild! Topology with be ignored.", td.getName());
-                    this.restoreCheckpointSchedulingState(schedulingState);
-                    //since state is restored need the update User topologySubmitter to the new User object in userMap
-                    topologySubmitter = this.userMap.get(td.getTopologySubmitter());
+                    topologySubmitter = this.cleanup(schedulingState, td);
                     topologySubmitter.moveTopoFromPendingToInvalid(td, this.cluster);
                     break;
                 }
@@ -253,6 +233,12 @@ public class ResourceAwareScheduler implements IScheduler {
         }
     }
 
+    private User cleanup(SchedulingState schedulingState, TopologyDetails td) {
+        this.restoreCheckpointSchedulingState(schedulingState);
+        //since state is restored need the update User topologySubmitter to the new User object in userMap
+        return this.userMap.get(td.getTopologySubmitter());
+    }
+
     private boolean mkAssignment(TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> schedulerAssignmentMap) {
         if (schedulerAssignmentMap != null) {
             double requestedMemOnHeap = td.getTotalRequestedMemOnHeap();
@@ -328,8 +314,7 @@ public class ResourceAwareScheduler implements IScheduler {
         LOG.debug("userResourcePools: {}", userResourcePools);
 
         for (TopologyDetails td : topologies.getTopologies()) {
-            //Get user that submitted topology.  If topology submitter is null or empty string, the topologySubmitter
-            //will be set to anonymous
+
             String topologySubmitter = td.getTopologySubmitter();
             //additional safety check to make sure that topologySubmitter is going to be a valid value
             if (topologySubmitter == null || topologySubmitter.equals("")) {
@@ -362,17 +347,18 @@ public class ResourceAwareScheduler implements IScheduler {
     /**
      * Get resource guarantee configs
      *
-     * @return
+     * @return a map that contains resource guarantees of every user of the following format
+     * {userid->{resourceType->amountGuaranteed}}
      */
     private Map<String, Map<String, Double>> getUserResourcePools() {
         Object raw = this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS);
         Map<String, Map<String, Double>> ret = new HashMap<String, Map<String, Double>>();
 
         if (raw != null) {
-            for (Map.Entry<String, Map<String, Number>> UserPoolEntry : ((Map<String, Map<String, Number>>) raw).entrySet()) {
-                String user = UserPoolEntry.getKey();
+            for (Map.Entry<String, Map<String, Number>> userPoolEntry : ((Map<String, Map<String, Number>>) raw).entrySet()) {
+                String user = userPoolEntry.getKey();
                 ret.put(user, new HashMap<String, Double>());
-                for (Map.Entry<String, Number> resourceEntry : UserPoolEntry.getValue().entrySet()) {
+                for (Map.Entry<String, Number> resourceEntry : userPoolEntry.getValue().entrySet()) {
                     ret.get(user).put(resourceEntry.getKey(), resourceEntry.getValue().doubleValue());
                 }
             }
@@ -381,10 +367,10 @@ public class ResourceAwareScheduler implements IScheduler {
         Map fromFile = Utils.findAndReadConfigFile("user-resource-pools.yaml", false);
         Map<String, Map<String, Number>> tmp = (Map<String, Map<String, Number>>) fromFile.get(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS);
         if (tmp != null) {
-            for (Map.Entry<String, Map<String, Number>> UserPoolEntry : tmp.entrySet()) {
-                String user = UserPoolEntry.getKey();
+            for (Map.Entry<String, Map<String, Number>> userPoolEntry : tmp.entrySet()) {
+                String user = userPoolEntry.getKey();
                 ret.put(user, new HashMap<String, Double>());
-                for (Map.Entry<String, Number> resourceEntry : UserPoolEntry.getValue().entrySet()) {
+                for (Map.Entry<String, Number> resourceEntry : userPoolEntry.getValue().entrySet()) {
                     ret.get(user).put(resourceEntry.getKey(), resourceEntry.getValue().doubleValue());
                 }
             }

http://git-wip-us.apache.org/repos/asf/storm/blob/0f80d067/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java
index 02d48e1..870b6c2 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java
@@ -152,7 +152,7 @@ public class ResourceUtils {
                     WorkerSlot slot = entry.getValue();
                     String nodeId = slot.getNodeId();
                     ExecutorDetails exec = entry.getKey();
-                    if (schedulingMap.containsKey(nodeId) == false) {
+                    if (!schedulingMap.containsKey(nodeId)) {
                         schedulingMap.put(nodeId, new HashMap<String, Map<WorkerSlot, Collection<ExecutorDetails>>>());
                     }
                     if (schedulingMap.get(nodeId).containsKey(topo.getId()) == false) {
@@ -181,13 +181,4 @@ public class ResourceUtils {
         }
         return str.toString();
     }
-
-    public static String printScheduling(RAS_Nodes nodes) {
-        String ret="";
-        for (RAS_Node node : nodes.getNodes()) {
-            ret += "Node: " + node.getHostname() + "\n";
-            ret += "-> " + node.getTopoIdTousedSlots() + "\n";
-        }
-        return ret;
-    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/0f80d067/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingResult.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingResult.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingResult.java
index 13ed8ad..3ea1b2a 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingResult.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingResult.java
@@ -80,7 +80,7 @@ public class SchedulingResult {
     }
 
     public Map<WorkerSlot, Collection<ExecutorDetails>> getSchedulingResultMap() {
-        return schedulingResultMap;
+        return this.schedulingResultMap;
     }
 
     public boolean isSuccess() {

http://git-wip-us.apache.org/repos/asf/storm/blob/0f80d067/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
index 7f49446..f1d53c6 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
@@ -26,8 +26,6 @@ import org.slf4j.LoggerFactory;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
@@ -324,6 +322,7 @@ public class User {
 
     /**
      * Comparator that sorts topologies by priority and then by submission time
+     * First sort by Topology Priority, if there is a tie for topology priority, topology uptime is used to sort
      */
     static class PQsortByPriorityAndSubmittionTime implements Comparator<TopologyDetails> {
 

http://git-wip-us.apache.org/repos/asf/storm/blob/0f80d067/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
index 1812580..d54ec43 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
@@ -34,14 +34,12 @@ public class DefaultEvictionStrategy implements IEvictionStrategy {
     private static final Logger LOG = LoggerFactory
             .getLogger(DefaultEvictionStrategy.class);
 
-    private Topologies topologies;
     private Cluster cluster;
     private Map<String, User> userMap;
     private RAS_Nodes nodes;
 
     @Override
     public void prepare(Topologies topologies, Cluster cluster, Map<String, User> userMap, RAS_Nodes nodes) {
-        this.topologies = topologies;
         this.cluster = cluster;
         this.userMap = userMap;
         this.nodes = nodes;
@@ -51,14 +49,16 @@ public class DefaultEvictionStrategy implements IEvictionStrategy {
     public boolean makeSpaceForTopo(TopologyDetails td) {
         LOG.debug("attempting to make space for topo {} from user {}", td.getName(), td.getTopologySubmitter());
         User submitter = this.userMap.get(td.getTopologySubmitter());
-        if (submitter.getCPUResourceGuaranteed() == null || submitter.getMemoryResourceGuaranteed() == null) {
+        if (submitter.getCPUResourceGuaranteed() == null || submitter.getMemoryResourceGuaranteed() == null
+                || submitter.getCPUResourceGuaranteed() == 0.0 || submitter.getMemoryResourceGuaranteed() == 0.0) {
             return false;
         }
+
         double cpuNeeded = td.getTotalRequestedCpu() / submitter.getCPUResourceGuaranteed();
         double memoryNeeded = (td.getTotalRequestedMemOffHeap() + td.getTotalRequestedMemOnHeap()) / submitter.getMemoryResourceGuaranteed();
 
         User evictUser = this.findUserWithHighestAverageResourceUtilAboveGuarantee();
-        //user has enough resource under his or her resource guarantee to schedule topology
+        //check if user has enough resource under his or her resource guarantee to schedule topology
         if ((1.0 - submitter.getCPUResourcePoolUtilization()) >= cpuNeeded && (1.0 - submitter.getMemoryResourcePoolUtilization()) >= memoryNeeded) {
             if (evictUser != null) {
                 TopologyDetails topologyEvict = evictUser.getRunningTopologyWithLowestPriority();
@@ -67,7 +67,7 @@ public class DefaultEvictionStrategy implements IEvictionStrategy {
             }
         } else {
             if (evictUser != null) {
-                if ((evictUser.getResourcePoolAverageUtilization() - 1.0) > (cpuNeeded + (submitter.getResourcePoolAverageUtilization() - 1.0))) {
+                if ((evictUser.getResourcePoolAverageUtilization() - 1.0) > (((cpuNeeded + memoryNeeded) / 2) + (submitter.getResourcePoolAverageUtilization() - 1.0))) {
                     TopologyDetails topologyEvict = evictUser.getRunningTopologyWithLowestPriority();
                     evictTopology(topologyEvict);
                     return true;

http://git-wip-us.apache.org/repos/asf/storm/blob/0f80d067/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java
index 0d891ff..c168ab8 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java
@@ -32,17 +32,13 @@ public class DefaultSchedulingPriorityStrategy implements ISchedulingPriorityStr
     private static final Logger LOG = LoggerFactory
             .getLogger(DefaultSchedulingPriorityStrategy.class);
 
-    private Topologies topologies;
     private Cluster cluster;
     private Map<String, User> userMap;
-    private RAS_Nodes nodes;
 
     @Override
     public void prepare(Topologies topologies, Cluster cluster, Map<String, User> userMap, RAS_Nodes nodes) {
-        this.topologies = topologies;
         this.cluster = cluster;
         this.userMap = userMap;
-        this.nodes = nodes;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/storm/blob/0f80d067/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java
index a7ac5c9..83d1289 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java
@@ -18,15 +18,11 @@
 
 package backtype.storm.scheduler.resource.strategies.scheduling;
 
-import java.util.Collection;
 import java.util.Map;
 
 import backtype.storm.scheduler.Cluster;
 import backtype.storm.scheduler.Topologies;
-import backtype.storm.scheduler.ExecutorDetails;
 import backtype.storm.scheduler.TopologyDetails;
-import backtype.storm.scheduler.WorkerSlot;
-import backtype.storm.scheduler.resource.RAS_Node;
 import backtype.storm.scheduler.resource.RAS_Nodes;
 import backtype.storm.scheduler.resource.SchedulingResult;
 import backtype.storm.scheduler.resource.User;

http://git-wip-us.apache.org/repos/asf/storm/blob/0f80d067/storm-core/test/jvm/backtype/storm/TestConfigValidate.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/TestConfigValidate.java b/storm-core/test/jvm/backtype/storm/TestConfigValidate.java
index 048cc92..0d317c6 100644
--- a/storm-core/test/jvm/backtype/storm/TestConfigValidate.java
+++ b/storm-core/test/jvm/backtype/storm/TestConfigValidate.java
@@ -633,7 +633,6 @@ public class TestConfigValidate {
     @Test
     public void TestResourceAwareSchedulerUserPool() {
         TestConfig config = new TestConfig();
-        Collection<Object> passCases = new LinkedList<Object>();
         Collection<Object> failCases = new LinkedList<Object>();
 
         Map<String, Map<String, Integer>> passCase1 = new HashMap<String, Map<String, Integer>>();
@@ -648,12 +647,8 @@ public class TestConfigValidate {
         passCase1.get("derek").put("cpu", 30000);
         passCase1.get("derek").put("memory", 60148);
 
-        passCases.add(passCase1);
-
-        for (Object value : passCases) {
-            config.put(TestConfig.TEST_MAP_CONFIG_7, value);
-            ConfigValidation.validateFields(config, TestConfig.class);
-        }
+        config.put(TestConfig.TEST_MAP_CONFIG_7, (Object) passCase1);
+        ConfigValidation.validateFields(config, TestConfig.class);
 
         Map<String, Map<String, Integer>> failCase1 = new HashMap<String, Map<String, Integer>>();
         failCase1.put("jerry", new HashMap<String, Integer>());
@@ -664,9 +659,11 @@ public class TestConfigValidate {
         failCase1.get("jerry").put("memory", 20148);
         failCase1.get("bobby").put("cpu", 20000);
         failCase1.get("bobby").put("memory", 40148);
+        //this will fail the test since user derek does not have an entry for memory
         failCase1.get("derek").put("cpu", 30000);
 
         Map<String, Map<String, Integer>> failCase2 = new HashMap<String, Map<String, Integer>>();
+        //this will fail since jerry doesn't have either cpu or memory entries
         failCase2.put("jerry", new HashMap<String, Integer>());
         failCase2.put("bobby", new HashMap<String, Integer>());
         failCase2.put("derek", new HashMap<String, Integer>());
@@ -700,7 +697,7 @@ public class TestConfigValidate {
             config.put(TestConfig.TEST_MAP_CONFIG_8, value);
             ConfigValidation.validateFields(config, TestConfig.class);
         }
-
+        //will fail since backtype.storm.nimbus.NimbusInfo doesn't implement or extend backtype.storm.networktopography.DNSToSwitchMapping
         failCases.add("backtype.storm.nimbus.NimbusInfo");
         failCases.add(null);
         for (Object value : failCases) {

http://git-wip-us.apache.org/repos/asf/storm/blob/0f80d067/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
index f9b4cd6..f5e875a 100644
--- a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
+++ b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
@@ -22,14 +22,12 @@ import backtype.storm.Config;
 import backtype.storm.scheduler.Cluster;
 import backtype.storm.scheduler.ExecutorDetails;
 import backtype.storm.scheduler.INimbus;
-import backtype.storm.scheduler.IScheduler;
 import backtype.storm.scheduler.SchedulerAssignment;
 import backtype.storm.scheduler.SchedulerAssignmentImpl;
 import backtype.storm.scheduler.SupervisorDetails;
 import backtype.storm.scheduler.Topologies;
 import backtype.storm.scheduler.TopologyDetails;
 import backtype.storm.scheduler.WorkerSlot;
-import backtype.storm.utils.Time;
 import backtype.storm.utils.Utils;
 import backtype.storm.validation.ConfigValidation;
 import org.junit.Assert;
@@ -37,7 +35,6 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -45,21 +42,86 @@ import java.util.Set;
 
 public class TestResourceAwareScheduler {
 
-    private static final int NUM_SUPS = 20;
-    private static final int NUM_WORKERS_PER_SUP = 4;
     private final String TOPOLOGY_SUBMITTER = "jerry";
 
     private static final Logger LOG = LoggerFactory.getLogger(TestResourceAwareScheduler.class);
 
+    private static int currentTime = 1450418597;
+
     @Test
     public void TestReadInResourceAwareSchedulerUserPools() {
-
         Map fromFile = Utils.findAndReadConfigFile("user-resource-pools.yaml", false);
         LOG.info("fromFile: {}", fromFile);
         ConfigValidation.validateFields(fromFile);
     }
 
     @Test
+    public void TestSubmitUsersWithNoGuarantees() {
+        INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
+        Map<String, Number> resourceMap = new HashMap<String, Number>();
+        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 100.0);
+        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1000.0);
+        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(4, 4, resourceMap);
+        Config config = new Config();
+        config.putAll(Utils.readDefaultConfig());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY, backtype.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy.class.getName());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, backtype.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy.class.getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class.getName());
+        config.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 100.0);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 500);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 500);
+
+        Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
+        resourceUserPool.put("jerry", new HashMap<String, Number>());
+        resourceUserPool.get("jerry").put("cpu", 200.0);
+        resourceUserPool.get("jerry").put("memory", 2000.0);
+
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
+        Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), config);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
+
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 10);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 20);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
+
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 10);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 20);
+
+        Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
+        topoMap.put(topo1.getId(), topo1);
+        topoMap.put(topo2.getId(), topo2);
+        topoMap.put(topo3.getId(), topo3);
+        topoMap.put(topo4.getId(), topo4);
+        topoMap.put(topo5.getId(), topo5);
+
+        Topologies topologies = new Topologies(topoMap);
+
+        ResourceAwareScheduler rs = new ResourceAwareScheduler();
+
+        rs.prepare(config);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("assert topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 3, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("assert topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("bobby").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 1, rs.getUser("bobby").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+    }
+
+    @Test
     public void TestTopologySortedInCorrectOrder() {
         INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
         Map<String, Number> resourceMap = new HashMap<String, Number>();
@@ -89,11 +151,11 @@ public class TestResourceAwareScheduler {
 
         config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
 
-        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
-        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
-        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30);
-        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
-        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30);
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, currentTime - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, currentTime - 8, 30);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 5, 15, 1, 1, currentTime - 16, 30);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 5, 15, 1, 1, currentTime - 16, 20);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 5, 15, 1, 1, currentTime - 24, 30);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo1.getId(), topo1);
@@ -138,7 +200,7 @@ public class TestResourceAwareScheduler {
         LOG.info("{} - {}", topo.getName(), queue);
         Assert.assertEquals("check order", topo.getName(), "topo-2");
 
-        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 5, 15, 1, 1, Time.currentTimeSecs() - 30, 10);
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 5, 15, 1, 1, currentTime - 30, 10);
         topoMap.put(topo6.getId(), topo6);
 
         topologies = new Topologies(topoMap);
@@ -200,27 +262,27 @@ public class TestResourceAwareScheduler {
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
 
-        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
-        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 29);
-        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 29);
-        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
-        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 29);
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, currentTime - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, currentTime - 8, 29);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 5, 15, 1, 1, currentTime - 16, 29);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 5, 15, 1, 1, currentTime - 16, 20);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 5, 15, 1, 1, currentTime - 24, 29);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
 
-        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
-        TopologyDetails topo7 = TestUtilsForResourceAwareScheduler.getTopology("topo-7", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 29);
-        TopologyDetails topo8 = TestUtilsForResourceAwareScheduler.getTopology("topo-8", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 29);
-        TopologyDetails topo9 = TestUtilsForResourceAwareScheduler.getTopology("topo-9", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
-        TopologyDetails topo10 = TestUtilsForResourceAwareScheduler.getTopology("topo-10", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 29);
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 5, 15, 1, 1, currentTime - 2, 20);
+        TopologyDetails topo7 = TestUtilsForResourceAwareScheduler.getTopology("topo-7", config, 5, 15, 1, 1, currentTime - 8, 29);
+        TopologyDetails topo8 = TestUtilsForResourceAwareScheduler.getTopology("topo-8", config, 5, 15, 1, 1, currentTime - 16, 29);
+        TopologyDetails topo9 = TestUtilsForResourceAwareScheduler.getTopology("topo-9", config, 5, 15, 1, 1, currentTime - 16, 20);
+        TopologyDetails topo10 = TestUtilsForResourceAwareScheduler.getTopology("topo-10", config, 5, 15, 1, 1, currentTime - 24, 29);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
-        TopologyDetails topo11 = TestUtilsForResourceAwareScheduler.getTopology("topo-11", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
-        TopologyDetails topo12 = TestUtilsForResourceAwareScheduler.getTopology("topo-12", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 29);
-        TopologyDetails topo13 = TestUtilsForResourceAwareScheduler.getTopology("topo-13", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 29);
-        TopologyDetails topo14 = TestUtilsForResourceAwareScheduler.getTopology("topo-14", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
-        TopologyDetails topo15 = TestUtilsForResourceAwareScheduler.getTopology("topo-15", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 29);
+        TopologyDetails topo11 = TestUtilsForResourceAwareScheduler.getTopology("topo-11", config, 5, 15, 1, 1, currentTime - 2, 20);
+        TopologyDetails topo12 = TestUtilsForResourceAwareScheduler.getTopology("topo-12", config, 5, 15, 1, 1, currentTime - 8, 29);
+        TopologyDetails topo13 = TestUtilsForResourceAwareScheduler.getTopology("topo-13", config, 5, 15, 1, 1, currentTime - 16, 29);
+        TopologyDetails topo14 = TestUtilsForResourceAwareScheduler.getTopology("topo-14", config, 5, 15, 1, 1, currentTime - 16, 20);
+        TopologyDetails topo15 = TestUtilsForResourceAwareScheduler.getTopology("topo-15", config, 5, 15, 1, 1, currentTime - 24, 29);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo1.getId(), topo1);
@@ -286,8 +348,8 @@ public class TestResourceAwareScheduler {
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
 
-        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
-        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 29);
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, currentTime - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, currentTime - 8, 29);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo1.getId(), topo1);
@@ -313,9 +375,9 @@ public class TestResourceAwareScheduler {
     }
 
     /**
-     * The resources in the cluster is limited. In the first round of scheduling, all resources in the cluster is used.
-     * User jerry submits another toplogy.  Since user jerry has has his resource guarantees satisfied, and user bobby
-     * has exceeded his resource guarantee, topo-3 from user bobby should be eviced.
+     * The resources in the cluster are limited. In the first round of scheduling, all resources in the cluster is used.
+     * User jerry submits another toploogy.  Since user jerry has his resource guarantees satisfied, and user bobby
+     * has exceeded his resource guarantee, topo-3 from user bobby should be evicted.
      */
     @Test
     public void testEviction() {
@@ -350,18 +412,18 @@ public class TestResourceAwareScheduler {
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
 
-        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
-        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 10);
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 20);
 
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
 
-        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
-        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 10);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
-        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 29);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo1.getId(), topo1);
@@ -377,7 +439,7 @@ public class TestResourceAwareScheduler {
         rs.schedule(topologies, cluster);
 
         for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
-            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+            Assert.assertTrue("assert topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
         Assert.assertEquals("# of running topologies", 1, rs.getUser("jerry").getTopologiesRunning().size());
         Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
@@ -385,7 +447,7 @@ public class TestResourceAwareScheduler {
         Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
 
         for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
-            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+            Assert.assertTrue("assert topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
         Assert.assertEquals("# of running topologies", 1, rs.getUser("derek").getTopologiesRunning().size());
         Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
@@ -393,7 +455,7 @@ public class TestResourceAwareScheduler {
         Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
 
         for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
-            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+            Assert.assertTrue("assert topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
         Assert.assertEquals("# of running topologies", 2, rs.getUser("bobby").getTopologiesRunning().size());
         Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
@@ -407,7 +469,7 @@ public class TestResourceAwareScheduler {
         rs.schedule(topologies, cluster);
 
         for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
-            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+            Assert.assertTrue("assert topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
         Assert.assertEquals("# of running topologies", 2, rs.getUser("jerry").getTopologiesRunning().size());
         Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
@@ -415,7 +477,7 @@ public class TestResourceAwareScheduler {
         Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
 
         for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
-            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+            Assert.assertTrue("assert topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
         Assert.assertEquals("# of running topologies", 1, rs.getUser("derek").getTopologiesRunning().size());
         Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
@@ -423,16 +485,17 @@ public class TestResourceAwareScheduler {
         Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
 
         for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
-            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+            Assert.assertTrue("assert topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
         Assert.assertEquals("# of running topologies", 1, rs.getUser("bobby").getTopologiesRunning().size());
 
         for (TopologyDetails topo : rs.getUser("bobby").getTopologiesAttempted()) {
-            Assert.assertFalse("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+            Assert.assertFalse("assert topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
         Assert.assertEquals("# of attempted topologies", 1, rs.getUser("bobby").getTopologiesAttempted().size());
         Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
         Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+        Assert.assertEquals("correct topology to evict", "topo-3", rs.getUser("bobby").getTopologiesAttempted().iterator().next().getName());
     }
 
     @Test
@@ -464,17 +527,17 @@ public class TestResourceAwareScheduler {
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
 
-        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 2, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 2, 0, 1, 0, currentTime - 2, 10);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
 
-        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
-        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 10);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
-        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
-        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 29);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 29);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo2.getId(), topo2);
@@ -490,7 +553,7 @@ public class TestResourceAwareScheduler {
         rs.schedule(topologies, cluster);
 
         for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
-            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+            Assert.assertTrue("assert topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
         Assert.assertEquals("# of running topologies", 2, rs.getUser("derek").getTopologiesRunning().size());
         Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
@@ -498,7 +561,7 @@ public class TestResourceAwareScheduler {
         Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
 
         for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
-            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+            Assert.assertTrue("assert topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
         Assert.assertEquals("# of running topologies", 2, rs.getUser("bobby").getTopologiesRunning().size());
         Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
@@ -511,7 +574,7 @@ public class TestResourceAwareScheduler {
         rs.schedule(topologies, cluster);
 
         for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
-            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+            Assert.assertTrue("assert topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
         Assert.assertEquals("# of running topologies", 1, rs.getUser("jerry").getTopologiesRunning().size());
         Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
@@ -519,7 +582,7 @@ public class TestResourceAwareScheduler {
         Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
 
         for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
-            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+            Assert.assertTrue("assert topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
         Assert.assertEquals("# of running topologies", 2, rs.getUser("derek").getTopologiesRunning().size());
         Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
@@ -527,7 +590,7 @@ public class TestResourceAwareScheduler {
         Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
 
         for (TopologyDetails topo : rs.getUser("bobby").getTopologiesAttempted()) {
-            Assert.assertFalse("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+            Assert.assertFalse("assert topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
         Assert.assertEquals("# of attempted topologies", 2, rs.getUser("bobby").getTopologiesAttempted().size());
         Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
@@ -571,19 +634,19 @@ public class TestResourceAwareScheduler {
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
 
-        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
-        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
-        TopologyDetails topo7 = TestUtilsForResourceAwareScheduler.getTopology("topo-7", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 10);
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 10);
+        TopologyDetails topo7 = TestUtilsForResourceAwareScheduler.getTopology("topo-7", config, 1, 0, 1, 0, currentTime - 2, 10);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
 
-        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
-        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 10);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
-        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
-        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 15, 29);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 29);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, currentTime - 15, 29);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo2.getId(), topo2);
@@ -751,19 +814,18 @@ public class TestResourceAwareScheduler {
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
 
-        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
-        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
-        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
-        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
-
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 20);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 29);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 10);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
 
-        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 10);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
-        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 29);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo1.getId(), topo1);
@@ -874,8 +936,7 @@ public class TestResourceAwareScheduler {
     }
 
     /**
-     * If topologies from other users cannot be evicted to make space
-     * check if there is a topology with lower priority that can be evicted from the current user
+     * If users are above his or her guarantee, check if topology eviction works correct
      */
     @Test
     public void TestOverGuaranteeEviction() {
@@ -910,18 +971,18 @@ public class TestResourceAwareScheduler {
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
 
-        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
-        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 20);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
 
-        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
-        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 10);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 10);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
-        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
-        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 29);
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 10);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo1.getId(), topo1);
@@ -961,7 +1022,8 @@ public class TestResourceAwareScheduler {
         Assert.assertEquals("# of attempted topologies", 0, rs.getUser("bobby").getTopologiesAttempted().size());
 
         //user derek submits another topology into a full cluster
-        // topo6 should not be able to scheduled
+        //topo6 should not be able to scheduled intially, but since topo6 has higher priority than topo5
+        //topo5 will be evicted so that topo6 can be scheduled
         topoMap.put(topo6.getId(), topo6);
         topologies = new Topologies(topoMap);
         rs.schedule(topologies, cluster);
@@ -1051,7 +1113,6 @@ public class TestResourceAwareScheduler {
         resourceUserPool.get("bobby").put("cpu", 200.0);
         resourceUserPool.get("bobby").put("memory", 2000.0);
 
-
         resourceUserPool.put("derek", new HashMap<String, Number>());
         resourceUserPool.get("derek").put("cpu", 100.0);
         resourceUserPool.get("derek").put("memory", 1000.0);
@@ -1061,18 +1122,18 @@ public class TestResourceAwareScheduler {
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
 
-        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
-        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, currentTime - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 20);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
 
-        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
-        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 10);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 10);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
-        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
-        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 29);
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 10);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo1.getId(), topo1);

http://git-wip-us.apache.org/repos/asf/storm/blob/0f80d067/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
index dcd487f..1aa010b 100644
--- a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
+++ b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
@@ -22,7 +22,6 @@ import backtype.storm.Config;
 import backtype.storm.generated.Bolt;
 import backtype.storm.generated.SpoutSpec;
 import backtype.storm.generated.StormTopology;
-import backtype.storm.generated.TopologySummary;
 import backtype.storm.scheduler.ExecutorDetails;
 import backtype.storm.scheduler.INimbus;
 import backtype.storm.scheduler.IScheduler;
@@ -57,26 +56,25 @@ import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 public class TestUtilsForResourceAwareScheduler {
+    private static int currentTime = 1450418597;
+
     private static final Logger LOG = LoggerFactory.getLogger(TestUtilsForResourceAwareScheduler.class);
 
     public static List<TopologyDetails> getListOfTopologies(Config config) {
 
         List<TopologyDetails> topos = new LinkedList<TopologyDetails>();
 
-        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20));
-        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30));
-        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30));
-        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20));
-        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30));
-        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 0));
-        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-7", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 0));
-        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-8", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 15));
-        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-9", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 8));
-        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-10", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 9));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, currentTime - 2, 20));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, currentTime - 8, 30));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 5, 15, 1, 1, currentTime - 16, 30));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 5, 15, 1, 1, currentTime - 16, 20));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 5, 15, 1, 1, currentTime - 24, 30));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 5, 15, 1, 1, currentTime - 2, 0));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-7", config, 5, 15, 1, 1, currentTime - 8, 0));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-8", config, 5, 15, 1, 1, currentTime - 16, 15));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-9", config, 5, 15, 1, 1, currentTime - 16, 8));
+        topos.add(TestUtilsForResourceAwareScheduler.getTopology("topo-10", config, 5, 15, 1, 1, currentTime - 24, 9));
         return topos;
     }
 


[08/23] storm git commit: made scheduling, eviction, and priority strategies pluggable

Posted by da...@apache.org.
made scheduling, eviction, and priority strategies pluggable


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/4cd5efa3
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/4cd5efa3
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/4cd5efa3

Branch: refs/heads/master
Commit: 4cd5efa3c5114413cf82ced7d3041e258175a5c0
Parents: 9d3c864
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Tue Nov 24 16:36:36 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Fri Dec 4 13:07:10 2015 -0600

----------------------------------------------------------------------
 conf/defaults.yaml                              |   3 +
 storm-core/src/jvm/backtype/storm/Config.java   |  37 +-
 .../resource/ResourceAwareScheduler.java        | 191 ++++----
 .../backtype/storm/scheduler/resource/User.java |  19 -
 .../resource/strategies/IStrategy.java          |  38 --
 .../strategies/ResourceAwareStrategy.java       | 486 ------------------
 .../eviction/DefaultEvictionStrategy.java       | 115 +++++
 .../strategies/eviction/IEvictionStrategy.java  |  47 ++
 .../DefaultSchedulingPriorityStrategy.java      |  92 ++++
 .../priority/ISchedulingPriorityStrategy.java   |  41 ++
 .../DefaultResourceAwareStrategy.java           | 487 +++++++++++++++++++
 .../strategies/scheduling/IStrategy.java        |  43 ++
 .../storm/validation/ConfigValidation.java      |  25 +
 .../validation/ConfigValidationAnnotations.java |   9 +
 .../jvm/backtype/storm/TestConfigValidate.java  |  29 ++
 15 files changed, 1009 insertions(+), 653 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/4cd5efa3/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 2a99ba6..cef09d3 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -256,6 +256,9 @@ topology.component.resources.onheap.memory.mb: 128.0
 topology.component.resources.offheap.memory.mb: 0.0
 topology.component.cpu.pcore.percent: 10.0
 topology.worker.max.heap.size.mb: 768.0
+topology.scheduler.strategy: "backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy"
+resource.aware.scheduler.eviction.strategy: "backtype.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy"
+resource.aware.scheduler.priority.strategy: "backtype.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy"
 
 dev.zookeeper.path: "/tmp/dev-storm-zookeeper"
 

http://git-wip-us.apache.org/repos/asf/storm/blob/4cd5efa3/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index d9c8815..54af6fa 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -17,6 +17,9 @@
  */
 package backtype.storm;
 
+import backtype.storm.scheduler.resource.strategies.eviction.IEvictionStrategy;
+import backtype.storm.scheduler.resource.strategies.priority.ISchedulingPriorityStrategy;
+import backtype.storm.scheduler.resource.strategies.scheduling.IStrategy;
 import backtype.storm.serialization.IKryoDecorator;
 import backtype.storm.serialization.IKryoFactory;
 import backtype.storm.validation.ConfigValidationAnnotations.*;
@@ -194,7 +197,8 @@ public class Config extends HashMap<String, Object> {
      * rack names that correspond to the supervisors. This information is stored in Cluster.java, and
      * is used in the resource aware scheduler.
      */
-    @isString
+    @NotNull
+    @isImplementationOfClass(implementsClass = backtype.storm.networktopography.DNSToSwitchMapping.class)
     public static final String STORM_NETWORK_TOPOGRAPHY_PLUGIN = "storm.network.topography.plugin";
 
     /**
@@ -1482,6 +1486,13 @@ public class Config extends HashMap<String, Object> {
     public static final String TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB = "topology.worker.max.heap.size.mb";
 
     /**
+     * The strategy to use when scheduling a topology with Resource Aware Scheduler
+     */
+    @NotNull
+    @isImplementationOfClass(implementsClass = IStrategy.class)
+    public static final String TOPOLOGY_SCHEDULER_STRATEGY = "topology.scheduler.strategy";
+
+    /**
      * How many executors to spawn for ackers.
      *
      * <p>By not setting this variable or setting it as null, Storm will set the number of acker executors
@@ -1943,6 +1954,20 @@ public class Config extends HashMap<String, Object> {
     public static final String RESOURCE_AWARE_SCHEDULER_USER_POOLS = "resource.aware.scheduler.user.pools";
 
     /**
+     * The class that specifies the eviction strategy to use in ResourceAwareScheduler
+     */
+    @NotNull
+    @isImplementationOfClass(implementsClass = IEvictionStrategy.class)
+    public static final String RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY = "resource.aware.scheduler.eviction.strategy";
+
+    /**
+     * the class that specifies the scheduling priority strategy to use in ResourceAwareScheduler
+     */
+    @NotNull
+    @isImplementationOfClass(implementsClass = ISchedulingPriorityStrategy.class)
+    public static final String RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY = "resource.aware.scheduler.priority.strategy";
+
+    /**
      * The number of machines that should be used by this topology to isolate it from all others. Set storm.scheduler
      * to backtype.storm.scheduler.multitenant.MultitenantScheduler
      */
@@ -2201,4 +2226,14 @@ public class Config extends HashMap<String, Object> {
     public void setTopologyPriority(int priority) {
         this.put(Config.TOPOLOGY_PRIORITY, priority);
     }
+
+    /**
+     * Takes as input the scheduler class name.
+     * Currently only the Multitenant Scheduler and Resource Aware Scheduler are supported
+     */
+    public void setTopologyStrategy(Class<? extends IStrategy> clazz) {
+        if(clazz != null) {
+            this.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, clazz.getName());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/4cd5efa3/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
index 934858e..f5c8354 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
@@ -19,7 +19,9 @@
 package backtype.storm.scheduler.resource;
 
 import backtype.storm.Config;
-import backtype.storm.scheduler.SchedulerAssignment;
+import backtype.storm.scheduler.resource.strategies.eviction.IEvictionStrategy;
+import backtype.storm.scheduler.resource.strategies.priority.ISchedulingPriorityStrategy;
+import backtype.storm.scheduler.resource.strategies.scheduling.IStrategy;
 import backtype.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -30,7 +32,7 @@ import backtype.storm.scheduler.IScheduler;
 import backtype.storm.scheduler.Topologies;
 import backtype.storm.scheduler.TopologyDetails;
 import backtype.storm.scheduler.WorkerSlot;
-import backtype.storm.scheduler.resource.strategies.ResourceAwareStrategy;
+import backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy;
 
 import java.util.Collection;
 import java.util.HashMap;
@@ -98,87 +100,36 @@ public class ResourceAwareScheduler implements IScheduler {
 
         LOG.info("Nodes:\n{}", this.nodes);
 
-        LOG.info("getNextUser: {}", this.getNextUser());
+        //LOG.info("getNextUser: {}", this.getNextUser());
 
+        ISchedulingPriorityStrategy schedulingPrioritystrategy = null;
         while (true) {
             LOG.info("/*********** next scheduling iteration **************/");
 
-            User nextUser = this.getNextUser();
-            if (nextUser == null) {
+            if(schedulingPrioritystrategy == null) {
+                try {
+                    schedulingPrioritystrategy = (ISchedulingPriorityStrategy) Utils.newInstance((String) this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY));
+                } catch (RuntimeException e) {
+                    LOG.error("failed to create instance of priority strategy: {} with error: {}! No topology eviction will be done.",
+                            this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY), e.getMessage());
+                    break;
+                }
+            }
+            //need to re prepare since scheduling state might have been restored
+            schedulingPrioritystrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
+            //Call scheduling priority strategy
+            TopologyDetails td = schedulingPrioritystrategy.getNextTopologyToSchedule();
+            if(td == null) {
                 break;
             }
-            TopologyDetails td = nextUser.getNextTopologyToSchedule();
             scheduleTopology(td);
         }
+
         //since scheduling state might have been restored thus need to set the cluster and topologies.
         cluster = this.cluster;
         topologies = this.topologies;
     }
 
-    private boolean makeSpaceForTopo(TopologyDetails td) {
-        LOG.info("attempting to make space for topo {} from user {}", td.getName(), td.getTopologySubmitter());
-        User submitter = this.userMap.get(td.getTopologySubmitter());
-        if (submitter.getCPUResourceGuaranteed() == null || submitter.getMemoryResourceGuaranteed() == null) {
-            return false;
-        }
-
-        double cpuNeeded = td.getTotalRequestedCpu() / submitter.getCPUResourceGuaranteed();
-        double memoryNeeded = (td.getTotalRequestedMemOffHeap() + td.getTotalRequestedMemOnHeap()) / submitter.getMemoryResourceGuaranteed();
-
-        //user has enough resource under his or her resource guarantee to schedule topology
-        if ((1.0 - submitter.getCPUResourcePoolUtilization()) >= cpuNeeded && (1.0 - submitter.getMemoryResourcePoolUtilization()) >= memoryNeeded) {
-            User evictUser = this.findUserWithMostResourcesAboveGuarantee();
-            if (evictUser == null) {
-                LOG.info("Cannot make space for topology {} from user {}", td.getName(), submitter.getId());
-                submitter.moveTopoFromPendingToAttempted(td, this.cluster);
-
-                return false;
-            }
-            TopologyDetails topologyEvict = evictUser.getRunningTopologyWithLowestPriority();
-            LOG.info("topology to evict: {}", topologyEvict);
-            evictTopology(topologyEvict);
-
-            LOG.info("Resources After eviction:\n{}", this.nodes);
-
-            return true;
-        } else {
-
-            if ((1.0 - submitter.getCPUResourcePoolUtilization()) < cpuNeeded) {
-
-            }
-
-            if ((1.0 - submitter.getMemoryResourcePoolUtilization()) < memoryNeeded) {
-
-            }
-            return false;
-
-        }
-    }
-
-    private void evictTopology(TopologyDetails topologyEvict) {
-        Collection<WorkerSlot> workersToEvict = this.cluster.getUsedSlotsByTopologyId(topologyEvict.getId());
-        User submitter = this.userMap.get(topologyEvict.getTopologySubmitter());
-
-        LOG.info("Evicting Topology {} with workers: {}", topologyEvict.getName(), workersToEvict);
-        LOG.debug("From Nodes:\n{}", ResourceUtils.printScheduling(this.nodes));
-        this.nodes.freeSlots(workersToEvict);
-        submitter.moveTopoFromRunningToPending(topologyEvict, this.cluster);
-        LOG.info("check if topology unassigned: {}", this.cluster.getUsedSlotsByTopologyId(topologyEvict.getId()));
-    }
-
-    private User findUserWithMostResourcesAboveGuarantee() {
-        double most = 0.0;
-        User mostOverUser = null;
-        for (User user : this.userMap.values()) {
-            double over = user.getResourcePoolAverageUtilization() - 1.0;
-            if ((over > most) && (!user.getTopologiesRunning().isEmpty())) {
-                most = over;
-                mostOverUser = user;
-            }
-        }
-        return mostOverUser;
-    }
-
     public void scheduleTopology(TopologyDetails td) {
         User topologySubmitter = this.userMap.get(td.getTopologySubmitter());
         if (cluster.getUnassignedExecutors(td).size() > 0) {
@@ -190,9 +141,19 @@ public class ResourceAwareScheduler implements IScheduler {
             LOG.debug("From Nodes:\n{}", ResourceUtils.printScheduling(this.nodes));
 
             SchedulingState schedulingState = this.checkpointSchedulingState();
+            IStrategy RAStrategy = null;
+            try {
+                RAStrategy = (IStrategy) Utils.newInstance((String) td.getConf().get(Config.TOPOLOGY_SCHEDULER_STRATEGY));
+            } catch (RuntimeException e) {
+                LOG.error("failed to create instance of IStrategy: {} with error: {}! Topology {} will not be scheduled.",
+                        td.getName(), td.getConf().get(Config.TOPOLOGY_SCHEDULER_STRATEGY), e.getMessage());
+                topologySubmitter.moveTopoFromPendingToInvalid(td, this.cluster);
+                return;
+            }
+            IEvictionStrategy evictionStrategy = null;
             while (true) {
-                //Need to reinitialize ResourceAwareStrategy with cluster and topologies in case scheduling state was restored
-                ResourceAwareStrategy RAStrategy = new ResourceAwareStrategy(this.cluster, this.topologies);
+                //Need to re prepare scheduling strategy with cluster and topologies in case scheduling state was restored
+                RAStrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
                 SchedulingResult result = RAStrategy.schedule(td);
                 LOG.info("scheduling result: {}", result);
                 if (result.isValid()) {
@@ -201,8 +162,8 @@ public class ResourceAwareScheduler implements IScheduler {
                             if (mkAssignment(td, result.getSchedulingResultMap())) {
                                 topologySubmitter.moveTopoFromPendingToRunning(td, this.cluster);
                             } else {
-                                //resetAssignments(assignmentCheckpoint);
                                 this.restoreCheckpointSchedulingState(schedulingState);
+                                //since state is restored need the update User topologySubmitter to the new User object in userMap
                                 topologySubmitter = this.userMap.get(td.getTopologySubmitter());
                                 topologySubmitter.moveTopoFromPendingToAttempted(td, this.cluster);
                             }
@@ -218,7 +179,19 @@ public class ResourceAwareScheduler implements IScheduler {
                         break;
                     } else {
                         if (result.getStatus() == SchedulingStatus.FAIL_NOT_ENOUGH_RESOURCES) {
-                            if (!this.makeSpaceForTopo(td)) {
+                            if(evictionStrategy == null) {
+                                try {
+                                    evictionStrategy = (IEvictionStrategy) Utils.newInstance((String) this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY));
+                                } catch (RuntimeException e) {
+                                    LOG.error("failed to create instance of eviction strategy: {} with error: {}! No topology eviction will be done.",
+                                            this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY), e.getMessage());
+                                    topologySubmitter.moveTopoFromPendingToAttempted(td);
+                                    break;
+                                }
+                            }
+                            //need to re prepare since scheduling state might have been restored
+                            evictionStrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
+                            if (!evictionStrategy.makeSpaceForTopo(td)) {
                                 this.cluster.setStatus(td.getId(), result.getErrorMessage());
                                 this.restoreCheckpointSchedulingState(schedulingState);
                                 //since state is restored need the update User topologySubmitter to the new User object in userMap
@@ -324,41 +297,41 @@ public class ResourceAwareScheduler implements IScheduler {
         return this.userMap;
     }
 
-    public User getNextUser() {
-        Double least = Double.POSITIVE_INFINITY;
-        User ret = null;
-        for (User user : this.userMap.values()) {
-            LOG.info("getNextUser {}", user.getDetailedInfo());
-            LOG.info("hasTopologyNeedSchedule: {}", user.hasTopologyNeedSchedule());
-            if (user.hasTopologyNeedSchedule()) {
-                Double userResourcePoolAverageUtilization = user.getResourcePoolAverageUtilization();
-                if(ret!=null) {
-                    LOG.info("current: {}-{} compareUser: {}-{}", ret.getId(), least, user.getId(), userResourcePoolAverageUtilization);
-                    LOG.info("{} == {}: {}", least, userResourcePoolAverageUtilization, least == userResourcePoolAverageUtilization);
-                    LOG.info("{} == {}: {}", least, userResourcePoolAverageUtilization, (Math.abs(least - userResourcePoolAverageUtilization) < 0.0001));
-
-                }
-                if (least > userResourcePoolAverageUtilization) {
-                    ret = user;
-                    least = userResourcePoolAverageUtilization;
-                } else if (Math.abs(least - userResourcePoolAverageUtilization) < 0.0001) {
-                    double currentCpuPercentage = ret.getCPUResourceGuaranteed() / this.cluster.getClusterTotalCPUResource();
-                    double currentMemoryPercentage = ret.getMemoryResourceGuaranteed() / this.cluster.getClusterTotalMemoryResource();
-                    double currentAvgPercentage = (currentCpuPercentage + currentMemoryPercentage) / 2.0;
-
-                    double userCpuPercentage = user.getCPUResourceGuaranteed() / this.cluster.getClusterTotalCPUResource();
-                    double userMemoryPercentage = user.getMemoryResourceGuaranteed() / this.cluster.getClusterTotalMemoryResource();
-                    double userAvgPercentage = (userCpuPercentage + userMemoryPercentage) / 2.0;
-                    LOG.info("current: {}-{} compareUser: {}-{}", ret.getId(), currentAvgPercentage, user.getId(), userAvgPercentage);
-                    if (userAvgPercentage > currentAvgPercentage) {
-                        ret = user;
-                        least = userResourcePoolAverageUtilization;
-                    }
-                }
-            }
-        }
-        return ret;
-    }
+//    public User getNextUser() {
+//        Double least = Double.POSITIVE_INFINITY;
+//        User ret = null;
+//        for (User user : this.userMap.values()) {
+//            LOG.info("getNextUser {}", user.getDetailedInfo());
+//            LOG.info("hasTopologyNeedSchedule: {}", user.hasTopologyNeedSchedule());
+//            if (user.hasTopologyNeedSchedule()) {
+//                Double userResourcePoolAverageUtilization = user.getResourcePoolAverageUtilization();
+//                if(ret!=null) {
+//                    LOG.info("current: {}-{} compareUser: {}-{}", ret.getId(), least, user.getId(), userResourcePoolAverageUtilization);
+//                    LOG.info("{} == {}: {}", least, userResourcePoolAverageUtilization, least == userResourcePoolAverageUtilization);
+//                    LOG.info("{} == {}: {}", least, userResourcePoolAverageUtilization, (Math.abs(least - userResourcePoolAverageUtilization) < 0.0001));
+//
+//                }
+//                if (least > userResourcePoolAverageUtilization) {
+//                    ret = user;
+//                    least = userResourcePoolAverageUtilization;
+//                } else if (Math.abs(least - userResourcePoolAverageUtilization) < 0.0001) {
+//                    double currentCpuPercentage = ret.getCPUResourceGuaranteed() / this.cluster.getClusterTotalCPUResource();
+//                    double currentMemoryPercentage = ret.getMemoryResourceGuaranteed() / this.cluster.getClusterTotalMemoryResource();
+//                    double currentAvgPercentage = (currentCpuPercentage + currentMemoryPercentage) / 2.0;
+//
+//                    double userCpuPercentage = user.getCPUResourceGuaranteed() / this.cluster.getClusterTotalCPUResource();
+//                    double userMemoryPercentage = user.getMemoryResourceGuaranteed() / this.cluster.getClusterTotalMemoryResource();
+//                    double userAvgPercentage = (userCpuPercentage + userMemoryPercentage) / 2.0;
+//                    LOG.info("current: {}-{} compareUser: {}-{}", ret.getId(), currentAvgPercentage, user.getId(), userAvgPercentage);
+//                    if (userAvgPercentage > currentAvgPercentage) {
+//                        ret = user;
+//                        least = userResourcePoolAverageUtilization;
+//                    }
+//                }
+//            }
+//        }
+//        return ret;
+//    }
 
     /**
      * Intialize scheduling and running queues

http://git-wip-us.apache.org/repos/asf/storm/blob/4cd5efa3/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
index 77a1dff..2d7c79f 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
@@ -314,25 +314,6 @@ public class User {
         return ret;
     }
 
-    public static int cTo(TopologyDetails topo1, TopologyDetails topo2) {
-        if (topo1.getId().compareTo(topo2.getId()) == 0) {
-            return 0;
-        }
-        if (topo1.getTopologyPriority() > topo2.getTopologyPriority()) {
-            return 1;
-        } else if (topo1.getTopologyPriority() < topo2.getTopologyPriority()) {
-            return -1;
-        } else {
-            if (topo1.getUpTime() > topo2.getUpTime()) {
-                return -1;
-            } else if (topo1.getUpTime() < topo2.getUpTime()) {
-                return 1;
-            } else {
-                return topo1.getId().compareTo(topo2.getId());
-            }
-        }
-    }
-
     /**
      * Comparator that sorts topologies by priority and then by submission time
      */

http://git-wip-us.apache.org/repos/asf/storm/blob/4cd5efa3/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/IStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/IStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/IStrategy.java
deleted file mode 100644
index 722eddb..0000000
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/IStrategy.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <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 backtype.storm.scheduler.resource.strategies;
-
-import java.util.Collection;
-import java.util.Map;
-
-import backtype.storm.scheduler.Topologies;
-import backtype.storm.scheduler.ExecutorDetails;
-import backtype.storm.scheduler.TopologyDetails;
-import backtype.storm.scheduler.WorkerSlot;
-import backtype.storm.scheduler.resource.RAS_Node;
-import backtype.storm.scheduler.resource.SchedulingResult;
-
-/**
- * An interface to for implementing different scheduling strategies for the resource aware scheduling
- * In the future stategies will be pluggable
- */
-public interface IStrategy {
-
-    public SchedulingResult schedule(TopologyDetails td);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/4cd5efa3/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/ResourceAwareStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/ResourceAwareStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/ResourceAwareStrategy.java
deleted file mode 100644
index 812bf5d..0000000
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/ResourceAwareStrategy.java
+++ /dev/null
@@ -1,486 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package backtype.storm.scheduler.resource.strategies;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Queue;
-import java.util.TreeMap;
-import java.util.HashSet;
-import java.util.Iterator;
-
-import backtype.storm.scheduler.resource.RAS_Nodes;
-import backtype.storm.scheduler.resource.SchedulingResult;
-import backtype.storm.scheduler.resource.SchedulingStatus;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import backtype.storm.scheduler.Cluster;
-import backtype.storm.scheduler.ExecutorDetails;
-import backtype.storm.scheduler.Topologies;
-import backtype.storm.scheduler.TopologyDetails;
-import backtype.storm.scheduler.WorkerSlot;
-import backtype.storm.scheduler.resource.Component;
-import backtype.storm.scheduler.resource.RAS_Node;
-
-public class ResourceAwareStrategy implements IStrategy {
-    private static final Logger LOG = LoggerFactory.getLogger(ResourceAwareStrategy.class);
-    private Topologies _topologies;
-    private Cluster _cluster;
-    //Map key is the supervisor id and the value is the corresponding RAS_Node Object 
-    private Map<String, RAS_Node> _availNodes;
-    private RAS_Node refNode = null;
-    /**
-     * supervisor id -> Node
-     */
-    private Map<String, RAS_Node> _nodes;
-    private Map<String, List<String>> _clusterInfo;
-
-    private final double CPU_WEIGHT = 1.0;
-    private final double MEM_WEIGHT = 1.0;
-    private final double NETWORK_WEIGHT = 1.0;
-
-    public ResourceAwareStrategy(Cluster cluster, Topologies topologies) {
-        _topologies = topologies;
-        _cluster = cluster;
-        _nodes = RAS_Nodes.getAllNodesFrom(cluster, _topologies);
-        _availNodes = this.getAvailNodes();
-        _clusterInfo = cluster.getNetworkTopography();
-        LOG.debug(this.getClusterInfo());
-    }
-
-    //the returned TreeMap keeps the Components sorted
-    private TreeMap<Integer, List<ExecutorDetails>> getPriorityToExecutorDetailsListMap(
-            Queue<Component> ordered__Component_list, Collection<ExecutorDetails> unassignedExecutors) {
-        TreeMap<Integer, List<ExecutorDetails>> retMap = new TreeMap<>();
-        Integer rank = 0;
-        for (Component ras_comp : ordered__Component_list) {
-            retMap.put(rank, new ArrayList<ExecutorDetails>());
-            for(ExecutorDetails exec : ras_comp.execs) {
-                if(unassignedExecutors.contains(exec)) {
-                    retMap.get(rank).add(exec);
-                }
-            }
-            rank++;
-        }
-        return retMap;
-    }
-
-    public SchedulingResult schedule(TopologyDetails td) {
-        if (_availNodes.size() <= 0) {
-            LOG.warn("No available nodes to schedule tasks on!");
-            return SchedulingResult.failure(SchedulingStatus.FAIL_NOT_ENOUGH_RESOURCES, "No available nodes to schedule tasks on!");
-        }
-        Collection<ExecutorDetails> unassignedExecutors = _cluster.getUnassignedExecutors(td);
-        Map<WorkerSlot, Collection<ExecutorDetails>> schedulerAssignmentMap = new HashMap<>();
-        LOG.debug("ExecutorsNeedScheduling: {}", unassignedExecutors);
-        Collection<ExecutorDetails> scheduledTasks = new ArrayList<>();
-        List<Component> spouts = this.getSpouts(_topologies, td);
-
-        if (spouts.size() == 0) {
-            LOG.error("Cannot find a Spout!");
-            return SchedulingResult.failure(SchedulingStatus.FAIL_INVALID_TOPOLOGY, "Cannot find a Spout!");
-        }
-
-        Queue<Component> ordered__Component_list = bfs(_topologies, td, spouts);
-
-        Map<Integer, List<ExecutorDetails>> priorityToExecutorMap = getPriorityToExecutorDetailsListMap(ordered__Component_list, unassignedExecutors);
-        Collection<ExecutorDetails> executorsNotScheduled = new HashSet<>(unassignedExecutors);
-        Integer longestPriorityListSize = this.getLongestPriorityListSize(priorityToExecutorMap);
-        //Pick the first executor with priority one, then the 1st exec with priority 2, so on an so forth. 
-        //Once we reach the last priority, we go back to priority 1 and schedule the second task with priority 1.
-        for (int i = 0; i < longestPriorityListSize; i++) {
-            for (Entry<Integer, List<ExecutorDetails>> entry : priorityToExecutorMap.entrySet()) {
-                Iterator<ExecutorDetails> it = entry.getValue().iterator();
-                if (it.hasNext()) {
-                    ExecutorDetails exec = it.next();
-                    LOG.debug("\n\nAttempting to schedule: {} of component {}[avail {}] with rank {}",
-                            new Object[] { exec, td.getExecutorToComponent().get(exec),
-                    td.getTaskResourceReqList(exec), entry.getKey() });
-                    WorkerSlot targetSlot = this.findWorkerForExec(exec, td, schedulerAssignmentMap);
-                    if (targetSlot != null) {
-                        RAS_Node targetNode = this.idToNode(targetSlot.getNodeId());
-                        if(!schedulerAssignmentMap.containsKey(targetSlot)) {
-                            schedulerAssignmentMap.put(targetSlot, new LinkedList<ExecutorDetails>());
-                        }
-                       
-                        schedulerAssignmentMap.get(targetSlot).add(exec);
-                        targetNode.consumeResourcesforTask(exec, td);
-                        scheduledTasks.add(exec);
-                        LOG.debug("TASK {} assigned to Node: {} avail [mem: {} cpu: {}] total [mem: {} cpu: {}] on slot: {}", exec,
-                                targetNode, targetNode.getAvailableMemoryResources(),
-                                targetNode.getAvailableCpuResources(), targetNode.getTotalMemoryResources(),
-                                targetNode.getTotalCpuResources(), targetSlot);
-                    } else {
-                        LOG.error("Not Enough Resources to schedule Task {}", exec);
-                    }
-                    it.remove();
-                }
-            }
-        }
-
-        executorsNotScheduled.removeAll(scheduledTasks);
-        LOG.debug("/* Scheduling left over task (most likely sys tasks) */");
-        // schedule left over system tasks
-        for (ExecutorDetails exec : executorsNotScheduled) {
-            WorkerSlot targetSlot = this.findWorkerForExec(exec, td, schedulerAssignmentMap);
-            if (targetSlot != null) {
-                RAS_Node targetNode = this.idToNode(targetSlot.getNodeId());
-                if(!schedulerAssignmentMap.containsKey(targetSlot)) {
-                    schedulerAssignmentMap.put(targetSlot, new LinkedList<ExecutorDetails>());
-                }
-               
-                schedulerAssignmentMap.get(targetSlot).add(exec);
-                targetNode.consumeResourcesforTask(exec, td);
-                scheduledTasks.add(exec);
-                LOG.debug("TASK {} assigned to Node: {} avail [mem: {} cpu: {}] total [mem: {} cpu: {}] on slot: {}", exec,
-                        targetNode, targetNode.getAvailableMemoryResources(),
-                        targetNode.getAvailableCpuResources(), targetNode.getTotalMemoryResources(),
-                        targetNode.getTotalCpuResources(), targetSlot);
-            } else {
-                LOG.error("Not Enough Resources to schedule Task {}", exec);
-            }
-        }
-
-        SchedulingResult result;
-        executorsNotScheduled.removeAll(scheduledTasks);
-        if (executorsNotScheduled.size() > 0) {
-            LOG.error("Not all executors successfully scheduled: {}",
-                    executorsNotScheduled);
-            schedulerAssignmentMap = null;
-            result = SchedulingResult.failure(SchedulingStatus.FAIL_NOT_ENOUGH_RESOURCES, "Not all executors successfully scheduled: " + executorsNotScheduled);
-        } else {
-            LOG.debug("All resources successfully scheduled!");
-            result = SchedulingResult.success(schedulerAssignmentMap);
-        }
-        if (schedulerAssignmentMap == null) {
-            LOG.error("Topology {} not successfully scheduled!", td.getId());
-        }
-        return result;
-    }
-
-    private WorkerSlot findWorkerForExec(ExecutorDetails exec, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
-      WorkerSlot ws;
-      // first scheduling
-      if (this.refNode == null) {
-          String clus = this.getBestClustering();
-          ws = this.getBestWorker(exec, td, clus, scheduleAssignmentMap);
-      } else {
-          ws = this.getBestWorker(exec, td, scheduleAssignmentMap);
-      }
-      if(ws != null) {
-          this.refNode = this.idToNode(ws.getNodeId());
-      }
-      LOG.debug("reference node for the resource aware scheduler is: {}", this.refNode);
-      return ws;
-    }
-
-    private WorkerSlot getBestWorker(ExecutorDetails exec, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
-        return this.getBestWorker(exec, td, null, scheduleAssignmentMap);
-    }
-
-    private WorkerSlot getBestWorker(ExecutorDetails exec, TopologyDetails td, String clusterId, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
-        double taskMem = td.getTotalMemReqTask(exec);
-        double taskCPU = td.getTotalCpuReqTask(exec);
-        List<RAS_Node> nodes;
-        if(clusterId != null) {
-            nodes = this.getAvailableNodesFromCluster(clusterId);
-            
-        } else {
-            nodes = this.getAvailableNodes();
-        }
-        //First sort nodes by distance
-        TreeMap<Double, RAS_Node> nodeRankMap = new TreeMap<>();
-        for (RAS_Node n : nodes) {
-            if(n.getFreeSlots().size()>0) {
-                if (n.getAvailableMemoryResources() >= taskMem
-                        && n.getAvailableCpuResources() >= taskCPU) {
-                    double a = Math.pow(((taskCPU - n.getAvailableCpuResources())/(n.getAvailableCpuResources() + 1))
-                            * this.CPU_WEIGHT, 2);
-                    double b = Math.pow(((taskMem - n.getAvailableMemoryResources())/(n.getAvailableMemoryResources() + 1))
-                            * this.MEM_WEIGHT, 2);
-                    double c = 0.0;
-                    if(this.refNode != null) {
-                        c = Math.pow(this.distToNode(this.refNode, n)
-                                * this.NETWORK_WEIGHT, 2);
-                    }
-                    double distance = Math.sqrt(a + b + c);
-                    nodeRankMap.put(distance, n);
-                }
-            }
-        }
-        //Then, pick worker from closest node that satisfy constraints
-        for(Map.Entry<Double, RAS_Node> entry : nodeRankMap.entrySet()) {
-            RAS_Node n = entry.getValue();
-            for(WorkerSlot ws : n.getFreeSlots()) {
-                if(checkWorkerConstraints(exec, ws, td, scheduleAssignmentMap)) {
-                    return ws;
-                }
-            }
-        }
-        return null;
-    }
-
-    private String getBestClustering() {
-        String bestCluster = null;
-        Double mostRes = 0.0;
-        for (Entry<String, List<String>> cluster : _clusterInfo
-                .entrySet()) {
-            Double clusterTotalRes = this.getTotalClusterRes(cluster.getValue());
-            if (clusterTotalRes > mostRes) {
-                mostRes = clusterTotalRes;
-                bestCluster = cluster.getKey();
-            }
-        }
-        return bestCluster;
-    }
-
-    private Double getTotalClusterRes(List<String> cluster) {
-        Double res = 0.0;
-        for (String node : cluster) {
-            res += _availNodes.get(this.NodeHostnameToId(node))
-                    .getAvailableMemoryResources()
-                    + _availNodes.get(this.NodeHostnameToId(node))
-                    .getAvailableCpuResources();
-        }
-        return res;
-    }
-
-    private Double distToNode(RAS_Node src, RAS_Node dest) {
-        if (src.getId().equals(dest.getId())) {
-            return 0.0;
-        } else if (this.NodeToCluster(src).equals(this.NodeToCluster(dest))) {
-            return 0.5;
-        } else {
-            return 1.0;
-        }
-    }
-
-    private String NodeToCluster(RAS_Node node) {
-        for (Entry<String, List<String>> entry : _clusterInfo
-                .entrySet()) {
-            if (entry.getValue().contains(node.getHostname())) {
-                return entry.getKey();
-            }
-        }
-        LOG.error("Node: {} not found in any clusters", node.getHostname());
-        return null;
-    }
-    
-    private List<RAS_Node> getAvailableNodes() {
-        LinkedList<RAS_Node> nodes = new LinkedList<>();
-        for (String clusterId : _clusterInfo.keySet()) {
-            nodes.addAll(this.getAvailableNodesFromCluster(clusterId));
-        }
-        return nodes;
-    }
-
-    private List<RAS_Node> getAvailableNodesFromCluster(String clus) {
-        List<RAS_Node> retList = new ArrayList<>();
-        for (String node_id : _clusterInfo.get(clus)) {
-            retList.add(_availNodes.get(this
-                    .NodeHostnameToId(node_id)));
-        }
-        return retList;
-    }
-
-    private List<WorkerSlot> getAvailableWorkersFromCluster(String clusterId) {
-        List<RAS_Node> nodes = this.getAvailableNodesFromCluster(clusterId);
-        List<WorkerSlot> workers = new LinkedList<>();
-        for(RAS_Node node : nodes) {
-            workers.addAll(node.getFreeSlots());
-        }
-        return workers;
-    }
-
-    private List<WorkerSlot> getAvailableWorker() {
-        List<WorkerSlot> workers = new LinkedList<>();
-        for (String clusterId : _clusterInfo.keySet()) {
-            workers.addAll(this.getAvailableWorkersFromCluster(clusterId));
-        }
-        return workers;
-    }
-
-    /**
-     * In case in the future RAS can only use a subset of nodes
-     */
-    private Map<String, RAS_Node> getAvailNodes() {
-        return _nodes;
-    }
-
-    /**
-     * Breadth first traversal of the topology DAG
-     * @param topologies
-     * @param td
-     * @param spouts
-     * @return A partial ordering of components
-     */
-    private Queue<Component> bfs(Topologies topologies, TopologyDetails td, List<Component> spouts) {
-        // Since queue is a interface
-        Queue<Component> ordered__Component_list = new LinkedList<Component>();
-        HashMap<String, Component> visited = new HashMap<>();
-
-        /* start from each spout that is not visited, each does a breadth-first traverse */
-        for (Component spout : spouts) {
-            if (!visited.containsKey(spout.id)) {
-                Queue<Component> queue = new LinkedList<>();
-                queue.offer(spout);
-                while (!queue.isEmpty()) {
-                    Component comp = queue.poll();
-                    visited.put(comp.id, comp);
-                    ordered__Component_list.add(comp);
-                    List<String> neighbors = new ArrayList<>();
-                    neighbors.addAll(comp.children);
-                    neighbors.addAll(comp.parents);
-                    for (String nbID : neighbors) {
-                        if (!visited.containsKey(nbID)) {
-                            Component child = topologies.getAllComponents().get(td.getId()).get(nbID);
-                            queue.offer(child);
-                        }
-                    }
-                }
-            }
-        }
-        return ordered__Component_list;
-    }
-
-    private List<Component> getSpouts(Topologies topologies, TopologyDetails td) {
-        List<Component> spouts = new ArrayList<>();
-        for (Component c : topologies.getAllComponents().get(td.getId())
-                .values()) {
-            if (c.type == Component.ComponentType.SPOUT) {
-                spouts.add(c);
-            }
-        }
-        return spouts;
-    }
-
-    private Integer getLongestPriorityListSize(Map<Integer, List<ExecutorDetails>> priorityToExecutorMap) {
-        Integer mostNum = 0;
-        for (List<ExecutorDetails> execs : priorityToExecutorMap.values()) {
-            Integer numExecs = execs.size();
-            if (mostNum < numExecs) {
-                mostNum = numExecs;
-            }
-        }
-        return mostNum;
-    }
-
-    /**
-     * Get the remaining amount memory that can be assigned to a worker given the set worker max heap size
-     * @param ws
-     * @param td
-     * @param scheduleAssignmentMap
-     * @return The remaining amount of memory
-     */
-    private Double getWorkerScheduledMemoryAvailable(WorkerSlot ws, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
-        Double memScheduleUsed = this.getWorkerScheduledMemoryUse(ws, td, scheduleAssignmentMap);
-        return td.getTopologyWorkerMaxHeapSize() - memScheduleUsed;
-    }
-
-    /**
-     * Get the amount of memory already assigned to a worker
-     * @param ws
-     * @param td
-     * @param scheduleAssignmentMap
-     * @return the amount of memory
-     */
-    private Double getWorkerScheduledMemoryUse(WorkerSlot ws, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
-        Double totalMem = 0.0;
-        Collection<ExecutorDetails> execs = scheduleAssignmentMap.get(ws);
-        if(execs != null) {
-            for(ExecutorDetails exec : execs) {
-                totalMem += td.getTotalMemReqTask(exec);
-            }
-        } 
-        return totalMem;
-    }
-
-    /**
-     * Checks whether we can schedule an Executor exec on the worker slot ws
-     * Only considers memory currently.  May include CPU in the future
-     * @param exec
-     * @param ws
-     * @param td
-     * @param scheduleAssignmentMap
-     * @return a boolean: True denoting the exec can be scheduled on ws and false if it cannot
-     */
-    private boolean checkWorkerConstraints(ExecutorDetails exec, WorkerSlot ws, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
-        boolean retVal = false;
-        if(this.getWorkerScheduledMemoryAvailable(ws, td, scheduleAssignmentMap) >= td.getTotalMemReqTask(exec)) {
-            retVal = true;
-        }
-        return retVal;
-    }
-
-    /**
-     * Get the amount of resources available and total for each node
-     * @return a String with cluster resource info for debug
-     */
-    private String getClusterInfo() {
-        String retVal = "Cluster info:\n";
-        for(Entry<String, List<String>> clusterEntry : _clusterInfo.entrySet()) {
-            String clusterId = clusterEntry.getKey();
-            retVal += "Rack: " + clusterId + "\n";
-            for(String nodeHostname : clusterEntry.getValue()) {
-                RAS_Node node = this.idToNode(this.NodeHostnameToId(nodeHostname));
-                retVal += "-> Node: " + node.getHostname() + " " + node.getId() + "\n";
-                retVal += "--> Avail Resources: {Mem " + node.getAvailableMemoryResources() + ", CPU " + node.getAvailableCpuResources() + "}\n";
-                retVal += "--> Total Resources: {Mem " + node.getTotalMemoryResources() + ", CPU " + node.getTotalCpuResources() + "}\n";
-            }
-        }
-        return retVal;
-    }
-
-    /**
-     * hostname to Id
-     * @param hostname
-     * @return the id of a node
-     */
-    public String NodeHostnameToId(String hostname) {
-        for (RAS_Node n : _nodes.values()) {
-            if (n.getHostname() == null) {
-                continue;
-            }
-            if (n.getHostname().equals(hostname)) {
-                return n.getId();
-            }
-        }
-        LOG.error("Cannot find Node with hostname {}", hostname);
-        return null;
-    }
-
-    /**
-     * Find RAS_Node for specified node id
-     * @param id
-     * @return a RAS_Node object
-     */
-    public RAS_Node idToNode(String id) {
-        if(_nodes.containsKey(id) == false) {
-            LOG.error("Cannot find Node with Id: {}", id);
-            return null;
-        }
-        return _nodes.get(id);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/4cd5efa3/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
new file mode 100644
index 0000000..7ca7ac3
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
@@ -0,0 +1,115 @@
+/**
+ * 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 backtype.storm.scheduler.resource.strategies.eviction;
+
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.scheduler.WorkerSlot;
+import backtype.storm.scheduler.resource.RAS_Nodes;
+import backtype.storm.scheduler.resource.ResourceUtils;
+import backtype.storm.scheduler.resource.User;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Map;
+
+public class DefaultEvictionStrategy implements IEvictionStrategy {
+    private static final Logger LOG = LoggerFactory
+            .getLogger(DefaultEvictionStrategy.class);
+
+    private Topologies topologies;
+    private Cluster cluster;
+    private Map<String, User> userMap;
+    private RAS_Nodes nodes;
+
+    @Override
+    public void prepare(Topologies topologies, Cluster cluster, Map<String, User> userMap, RAS_Nodes nodes) {
+        this.topologies = topologies;
+        this.cluster = cluster;
+        this.userMap = userMap;
+        this.nodes = nodes;
+    }
+
+    @Override
+    public boolean makeSpaceForTopo(TopologyDetails td) {
+        LOG.info("attempting to make space for topo {} from user {}", td.getName(), td.getTopologySubmitter());
+        User submitter = this.userMap.get(td.getTopologySubmitter());
+        if (submitter.getCPUResourceGuaranteed() == null || submitter.getMemoryResourceGuaranteed() == null) {
+            return false;
+        }
+
+        double cpuNeeded = td.getTotalRequestedCpu() / submitter.getCPUResourceGuaranteed();
+        double memoryNeeded = (td.getTotalRequestedMemOffHeap() + td.getTotalRequestedMemOnHeap()) / submitter.getMemoryResourceGuaranteed();
+
+        //user has enough resource under his or her resource guarantee to schedule topology
+        if ((1.0 - submitter.getCPUResourcePoolUtilization()) >= cpuNeeded && (1.0 - submitter.getMemoryResourcePoolUtilization()) >= memoryNeeded) {
+            User evictUser = this.findUserWithMostResourcesAboveGuarantee();
+            if (evictUser == null) {
+                LOG.info("Cannot make space for topology {} from user {}", td.getName(), submitter.getId());
+                submitter.moveTopoFromPendingToAttempted(td, this.cluster);
+
+                return false;
+            }
+            TopologyDetails topologyEvict = evictUser.getRunningTopologyWithLowestPriority();
+            LOG.info("topology to evict: {}", topologyEvict);
+            evictTopology(topologyEvict);
+
+            LOG.info("Resources After eviction:\n{}", this.nodes);
+
+            return true;
+        } else {
+
+            if ((1.0 - submitter.getCPUResourcePoolUtilization()) < cpuNeeded) {
+
+            }
+
+            if ((1.0 - submitter.getMemoryResourcePoolUtilization()) < memoryNeeded) {
+
+            }
+            return false;
+
+        }
+    }
+
+    private void evictTopology(TopologyDetails topologyEvict) {
+        Collection<WorkerSlot> workersToEvict = this.cluster.getUsedSlotsByTopologyId(topologyEvict.getId());
+        User submitter = this.userMap.get(topologyEvict.getTopologySubmitter());
+
+        LOG.info("Evicting Topology {} with workers: {}", topologyEvict.getName(), workersToEvict);
+        LOG.debug("From Nodes:\n{}", ResourceUtils.printScheduling(this.nodes));
+        this.nodes.freeSlots(workersToEvict);
+        submitter.moveTopoFromRunningToPending(topologyEvict, this.cluster);
+        LOG.info("check if topology unassigned: {}", this.cluster.getUsedSlotsByTopologyId(topologyEvict.getId()));
+    }
+
+    private User findUserWithMostResourcesAboveGuarantee() {
+        double most = 0.0;
+        User mostOverUser = null;
+        for (User user : this.userMap.values()) {
+            double over = user.getResourcePoolAverageUtilization() - 1.0;
+            if ((over > most) && (!user.getTopologiesRunning().isEmpty())) {
+                most = over;
+                mostOverUser = user;
+            }
+        }
+        return mostOverUser;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4cd5efa3/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/IEvictionStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/IEvictionStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/IEvictionStrategy.java
new file mode 100644
index 0000000..b787434
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/IEvictionStrategy.java
@@ -0,0 +1,47 @@
+/**
+ * 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 backtype.storm.scheduler.resource.strategies.eviction;
+
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.scheduler.resource.RAS_Nodes;
+import backtype.storm.scheduler.resource.User;
+
+import java.util.Map;
+
+public interface IEvictionStrategy {
+
+    /**
+     * Initialization
+     */
+    public void prepare(Topologies topologies, Cluster cluster, Map<String, User> userMap, RAS_Nodes nodes);
+
+    /**
+     * This method when invoked should attempt to make space on the cluster so that the topology specified can be scheduled
+     * @param td the topology to make space for
+     * @return return true to indicate that space has been made for topology and try schedule topology td again.
+     * Return false to inidcate that no space could be made for the topology on the cluster and the scheduler should give up
+     * trying to schedule the topology for this round of scheduling.  This method will be invoked until the topology indicated
+     * could be scheduled or the method returns false
+     */
+    public boolean makeSpaceForTopo(TopologyDetails td);
+
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4cd5efa3/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java
new file mode 100644
index 0000000..5096fd6
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java
@@ -0,0 +1,92 @@
+/**
+ * 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 backtype.storm.scheduler.resource.strategies.priority;
+
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.scheduler.resource.RAS_Nodes;
+import backtype.storm.scheduler.resource.User;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+public class DefaultSchedulingPriorityStrategy implements  ISchedulingPriorityStrategy{
+    private static final Logger LOG = LoggerFactory
+            .getLogger(DefaultSchedulingPriorityStrategy.class);
+
+    private Topologies topologies;
+    private Cluster cluster;
+    private Map<String, User> userMap;
+    private RAS_Nodes nodes;
+
+    @Override
+    public void prepare(Topologies topologies, Cluster cluster, Map<String, User> userMap, RAS_Nodes nodes) {
+        this.topologies = topologies;
+        this.cluster = cluster;
+        this.userMap = userMap;
+        this.nodes = nodes;
+    }
+
+    @Override
+    public TopologyDetails getNextTopologyToSchedule() {
+        User nextUser = this.getNextUser();
+        if (nextUser == null) {
+            return null;
+        }
+        return nextUser.getNextTopologyToSchedule();
+    }
+
+    public User getNextUser() {
+        Double least = Double.POSITIVE_INFINITY;
+        User ret = null;
+        for (User user : this.userMap.values()) {
+            LOG.info("getNextUser {}", user.getDetailedInfo());
+            LOG.info("hasTopologyNeedSchedule: {}", user.hasTopologyNeedSchedule());
+            if (user.hasTopologyNeedSchedule()) {
+                Double userResourcePoolAverageUtilization = user.getResourcePoolAverageUtilization();
+                if(ret!=null) {
+                    LOG.info("current: {}-{} compareUser: {}-{}", ret.getId(), least, user.getId(), userResourcePoolAverageUtilization);
+                    LOG.info("{} == {}: {}", least, userResourcePoolAverageUtilization, least == userResourcePoolAverageUtilization);
+                    LOG.info("{} == {}: {}", least, userResourcePoolAverageUtilization, (Math.abs(least - userResourcePoolAverageUtilization) < 0.0001));
+
+                }
+                if (least > userResourcePoolAverageUtilization) {
+                    ret = user;
+                    least = userResourcePoolAverageUtilization;
+                } else if (Math.abs(least - userResourcePoolAverageUtilization) < 0.0001) {
+                    double currentCpuPercentage = ret.getCPUResourceGuaranteed() / this.cluster.getClusterTotalCPUResource();
+                    double currentMemoryPercentage = ret.getMemoryResourceGuaranteed() / this.cluster.getClusterTotalMemoryResource();
+                    double currentAvgPercentage = (currentCpuPercentage + currentMemoryPercentage) / 2.0;
+
+                    double userCpuPercentage = user.getCPUResourceGuaranteed() / this.cluster.getClusterTotalCPUResource();
+                    double userMemoryPercentage = user.getMemoryResourceGuaranteed() / this.cluster.getClusterTotalMemoryResource();
+                    double userAvgPercentage = (userCpuPercentage + userMemoryPercentage) / 2.0;
+                    LOG.info("current: {}-{} compareUser: {}-{}", ret.getId(), currentAvgPercentage, user.getId(), userAvgPercentage);
+                    if (userAvgPercentage > currentAvgPercentage) {
+                        ret = user;
+                        least = userResourcePoolAverageUtilization;
+                    }
+                }
+            }
+        }
+        return ret;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4cd5efa3/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/ISchedulingPriorityStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/ISchedulingPriorityStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/ISchedulingPriorityStrategy.java
new file mode 100644
index 0000000..7e92b3d
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/ISchedulingPriorityStrategy.java
@@ -0,0 +1,41 @@
+/**
+ * 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 backtype.storm.scheduler.resource.strategies.priority;
+
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.scheduler.resource.RAS_Nodes;
+import backtype.storm.scheduler.resource.User;
+
+import java.util.Map;
+
+public interface ISchedulingPriorityStrategy {
+
+    /**
+     * initializes
+     */
+    public void prepare(Topologies topologies, Cluster cluster, Map<String, User> userMap, RAS_Nodes nodes);
+
+    /**
+     *
+     * @return return the next topology to schedule.  If there is no topologies left to schedule, return null
+     */
+    public TopologyDetails getNextTopologyToSchedule();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4cd5efa3/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/DefaultResourceAwareStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/DefaultResourceAwareStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/DefaultResourceAwareStrategy.java
new file mode 100644
index 0000000..1950858
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/DefaultResourceAwareStrategy.java
@@ -0,0 +1,487 @@
+/**
+ * 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 backtype.storm.scheduler.resource.strategies.scheduling;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Queue;
+import java.util.TreeMap;
+import java.util.HashSet;
+import java.util.Iterator;
+
+import backtype.storm.scheduler.resource.RAS_Nodes;
+import backtype.storm.scheduler.resource.SchedulingResult;
+import backtype.storm.scheduler.resource.SchedulingStatus;
+import backtype.storm.scheduler.resource.User;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.ExecutorDetails;
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.scheduler.WorkerSlot;
+import backtype.storm.scheduler.resource.Component;
+import backtype.storm.scheduler.resource.RAS_Node;
+
+public class DefaultResourceAwareStrategy implements IStrategy {
+    private static final Logger LOG = LoggerFactory.getLogger(DefaultResourceAwareStrategy.class);
+    private Topologies _topologies;
+    private Cluster _cluster;
+    //Map key is the supervisor id and the value is the corresponding RAS_Node Object 
+    private Map<String, RAS_Node> _availNodes;
+    private RAS_Node refNode = null;
+    /**
+     * supervisor id -> Node
+     */
+    private Map<String, RAS_Node> _nodes;
+    private Map<String, List<String>> _clusterInfo;
+
+    private final double CPU_WEIGHT = 1.0;
+    private final double MEM_WEIGHT = 1.0;
+    private final double NETWORK_WEIGHT = 1.0;
+
+    public void prepare (Topologies topologies, Cluster cluster, Map<String, User> userMap, RAS_Nodes nodes) {
+        _topologies = topologies;
+        _cluster = cluster;
+        _nodes = RAS_Nodes.getAllNodesFrom(cluster, _topologies);
+        _availNodes = this.getAvailNodes();
+        _clusterInfo = cluster.getNetworkTopography();
+        LOG.debug(this.getClusterInfo());
+    }
+
+    //the returned TreeMap keeps the Components sorted
+    private TreeMap<Integer, List<ExecutorDetails>> getPriorityToExecutorDetailsListMap(
+            Queue<Component> ordered__Component_list, Collection<ExecutorDetails> unassignedExecutors) {
+        TreeMap<Integer, List<ExecutorDetails>> retMap = new TreeMap<>();
+        Integer rank = 0;
+        for (Component ras_comp : ordered__Component_list) {
+            retMap.put(rank, new ArrayList<ExecutorDetails>());
+            for(ExecutorDetails exec : ras_comp.execs) {
+                if(unassignedExecutors.contains(exec)) {
+                    retMap.get(rank).add(exec);
+                }
+            }
+            rank++;
+        }
+        return retMap;
+    }
+
+    public SchedulingResult schedule(TopologyDetails td) {
+        if (_availNodes.size() <= 0) {
+            LOG.warn("No available nodes to schedule tasks on!");
+            return SchedulingResult.failure(SchedulingStatus.FAIL_NOT_ENOUGH_RESOURCES, "No available nodes to schedule tasks on!");
+        }
+        Collection<ExecutorDetails> unassignedExecutors = _cluster.getUnassignedExecutors(td);
+        Map<WorkerSlot, Collection<ExecutorDetails>> schedulerAssignmentMap = new HashMap<>();
+        LOG.debug("ExecutorsNeedScheduling: {}", unassignedExecutors);
+        Collection<ExecutorDetails> scheduledTasks = new ArrayList<>();
+        List<Component> spouts = this.getSpouts(_topologies, td);
+
+        if (spouts.size() == 0) {
+            LOG.error("Cannot find a Spout!");
+            return SchedulingResult.failure(SchedulingStatus.FAIL_INVALID_TOPOLOGY, "Cannot find a Spout!");
+        }
+
+        Queue<Component> ordered__Component_list = bfs(_topologies, td, spouts);
+
+        Map<Integer, List<ExecutorDetails>> priorityToExecutorMap = getPriorityToExecutorDetailsListMap(ordered__Component_list, unassignedExecutors);
+        Collection<ExecutorDetails> executorsNotScheduled = new HashSet<>(unassignedExecutors);
+        Integer longestPriorityListSize = this.getLongestPriorityListSize(priorityToExecutorMap);
+        //Pick the first executor with priority one, then the 1st exec with priority 2, so on an so forth. 
+        //Once we reach the last priority, we go back to priority 1 and schedule the second task with priority 1.
+        for (int i = 0; i < longestPriorityListSize; i++) {
+            for (Entry<Integer, List<ExecutorDetails>> entry : priorityToExecutorMap.entrySet()) {
+                Iterator<ExecutorDetails> it = entry.getValue().iterator();
+                if (it.hasNext()) {
+                    ExecutorDetails exec = it.next();
+                    LOG.debug("\n\nAttempting to schedule: {} of component {}[avail {}] with rank {}",
+                            new Object[] { exec, td.getExecutorToComponent().get(exec),
+                    td.getTaskResourceReqList(exec), entry.getKey() });
+                    WorkerSlot targetSlot = this.findWorkerForExec(exec, td, schedulerAssignmentMap);
+                    if (targetSlot != null) {
+                        RAS_Node targetNode = this.idToNode(targetSlot.getNodeId());
+                        if(!schedulerAssignmentMap.containsKey(targetSlot)) {
+                            schedulerAssignmentMap.put(targetSlot, new LinkedList<ExecutorDetails>());
+                        }
+                       
+                        schedulerAssignmentMap.get(targetSlot).add(exec);
+                        targetNode.consumeResourcesforTask(exec, td);
+                        scheduledTasks.add(exec);
+                        LOG.debug("TASK {} assigned to Node: {} avail [mem: {} cpu: {}] total [mem: {} cpu: {}] on slot: {}", exec,
+                                targetNode, targetNode.getAvailableMemoryResources(),
+                                targetNode.getAvailableCpuResources(), targetNode.getTotalMemoryResources(),
+                                targetNode.getTotalCpuResources(), targetSlot);
+                    } else {
+                        LOG.error("Not Enough Resources to schedule Task {}", exec);
+                    }
+                    it.remove();
+                }
+            }
+        }
+
+        executorsNotScheduled.removeAll(scheduledTasks);
+        LOG.debug("/* Scheduling left over task (most likely sys tasks) */");
+        // schedule left over system tasks
+        for (ExecutorDetails exec : executorsNotScheduled) {
+            WorkerSlot targetSlot = this.findWorkerForExec(exec, td, schedulerAssignmentMap);
+            if (targetSlot != null) {
+                RAS_Node targetNode = this.idToNode(targetSlot.getNodeId());
+                if(!schedulerAssignmentMap.containsKey(targetSlot)) {
+                    schedulerAssignmentMap.put(targetSlot, new LinkedList<ExecutorDetails>());
+                }
+               
+                schedulerAssignmentMap.get(targetSlot).add(exec);
+                targetNode.consumeResourcesforTask(exec, td);
+                scheduledTasks.add(exec);
+                LOG.debug("TASK {} assigned to Node: {} avail [mem: {} cpu: {}] total [mem: {} cpu: {}] on slot: {}", exec,
+                        targetNode, targetNode.getAvailableMemoryResources(),
+                        targetNode.getAvailableCpuResources(), targetNode.getTotalMemoryResources(),
+                        targetNode.getTotalCpuResources(), targetSlot);
+            } else {
+                LOG.error("Not Enough Resources to schedule Task {}", exec);
+            }
+        }
+
+        SchedulingResult result;
+        executorsNotScheduled.removeAll(scheduledTasks);
+        if (executorsNotScheduled.size() > 0) {
+            LOG.error("Not all executors successfully scheduled: {}",
+                    executorsNotScheduled);
+            schedulerAssignmentMap = null;
+            result = SchedulingResult.failure(SchedulingStatus.FAIL_NOT_ENOUGH_RESOURCES, "Not all executors successfully scheduled: " + executorsNotScheduled);
+        } else {
+            LOG.debug("All resources successfully scheduled!");
+            result = SchedulingResult.success(schedulerAssignmentMap);
+        }
+        if (schedulerAssignmentMap == null) {
+            LOG.error("Topology {} not successfully scheduled!", td.getId());
+        }
+        return result;
+    }
+
+    private WorkerSlot findWorkerForExec(ExecutorDetails exec, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
+      WorkerSlot ws;
+      // first scheduling
+      if (this.refNode == null) {
+          String clus = this.getBestClustering();
+          ws = this.getBestWorker(exec, td, clus, scheduleAssignmentMap);
+      } else {
+          ws = this.getBestWorker(exec, td, scheduleAssignmentMap);
+      }
+      if(ws != null) {
+          this.refNode = this.idToNode(ws.getNodeId());
+      }
+      LOG.debug("reference node for the resource aware scheduler is: {}", this.refNode);
+      return ws;
+    }
+
+    private WorkerSlot getBestWorker(ExecutorDetails exec, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
+        return this.getBestWorker(exec, td, null, scheduleAssignmentMap);
+    }
+
+    private WorkerSlot getBestWorker(ExecutorDetails exec, TopologyDetails td, String clusterId, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
+        double taskMem = td.getTotalMemReqTask(exec);
+        double taskCPU = td.getTotalCpuReqTask(exec);
+        List<RAS_Node> nodes;
+        if(clusterId != null) {
+            nodes = this.getAvailableNodesFromCluster(clusterId);
+            
+        } else {
+            nodes = this.getAvailableNodes();
+        }
+        //First sort nodes by distance
+        TreeMap<Double, RAS_Node> nodeRankMap = new TreeMap<>();
+        for (RAS_Node n : nodes) {
+            if(n.getFreeSlots().size()>0) {
+                if (n.getAvailableMemoryResources() >= taskMem
+                        && n.getAvailableCpuResources() >= taskCPU) {
+                    double a = Math.pow(((taskCPU - n.getAvailableCpuResources())/(n.getAvailableCpuResources() + 1))
+                            * this.CPU_WEIGHT, 2);
+                    double b = Math.pow(((taskMem - n.getAvailableMemoryResources())/(n.getAvailableMemoryResources() + 1))
+                            * this.MEM_WEIGHT, 2);
+                    double c = 0.0;
+                    if(this.refNode != null) {
+                        c = Math.pow(this.distToNode(this.refNode, n)
+                                * this.NETWORK_WEIGHT, 2);
+                    }
+                    double distance = Math.sqrt(a + b + c);
+                    nodeRankMap.put(distance, n);
+                }
+            }
+        }
+        //Then, pick worker from closest node that satisfy constraints
+        for(Map.Entry<Double, RAS_Node> entry : nodeRankMap.entrySet()) {
+            RAS_Node n = entry.getValue();
+            for(WorkerSlot ws : n.getFreeSlots()) {
+                if(checkWorkerConstraints(exec, ws, td, scheduleAssignmentMap)) {
+                    return ws;
+                }
+            }
+        }
+        return null;
+    }
+
+    private String getBestClustering() {
+        String bestCluster = null;
+        Double mostRes = 0.0;
+        for (Entry<String, List<String>> cluster : _clusterInfo
+                .entrySet()) {
+            Double clusterTotalRes = this.getTotalClusterRes(cluster.getValue());
+            if (clusterTotalRes > mostRes) {
+                mostRes = clusterTotalRes;
+                bestCluster = cluster.getKey();
+            }
+        }
+        return bestCluster;
+    }
+
+    private Double getTotalClusterRes(List<String> cluster) {
+        Double res = 0.0;
+        for (String node : cluster) {
+            res += _availNodes.get(this.NodeHostnameToId(node))
+                    .getAvailableMemoryResources()
+                    + _availNodes.get(this.NodeHostnameToId(node))
+                    .getAvailableCpuResources();
+        }
+        return res;
+    }
+
+    private Double distToNode(RAS_Node src, RAS_Node dest) {
+        if (src.getId().equals(dest.getId())) {
+            return 0.0;
+        } else if (this.NodeToCluster(src).equals(this.NodeToCluster(dest))) {
+            return 0.5;
+        } else {
+            return 1.0;
+        }
+    }
+
+    private String NodeToCluster(RAS_Node node) {
+        for (Entry<String, List<String>> entry : _clusterInfo
+                .entrySet()) {
+            if (entry.getValue().contains(node.getHostname())) {
+                return entry.getKey();
+            }
+        }
+        LOG.error("Node: {} not found in any clusters", node.getHostname());
+        return null;
+    }
+    
+    private List<RAS_Node> getAvailableNodes() {
+        LinkedList<RAS_Node> nodes = new LinkedList<>();
+        for (String clusterId : _clusterInfo.keySet()) {
+            nodes.addAll(this.getAvailableNodesFromCluster(clusterId));
+        }
+        return nodes;
+    }
+
+    private List<RAS_Node> getAvailableNodesFromCluster(String clus) {
+        List<RAS_Node> retList = new ArrayList<>();
+        for (String node_id : _clusterInfo.get(clus)) {
+            retList.add(_availNodes.get(this
+                    .NodeHostnameToId(node_id)));
+        }
+        return retList;
+    }
+
+    private List<WorkerSlot> getAvailableWorkersFromCluster(String clusterId) {
+        List<RAS_Node> nodes = this.getAvailableNodesFromCluster(clusterId);
+        List<WorkerSlot> workers = new LinkedList<>();
+        for(RAS_Node node : nodes) {
+            workers.addAll(node.getFreeSlots());
+        }
+        return workers;
+    }
+
+    private List<WorkerSlot> getAvailableWorker() {
+        List<WorkerSlot> workers = new LinkedList<>();
+        for (String clusterId : _clusterInfo.keySet()) {
+            workers.addAll(this.getAvailableWorkersFromCluster(clusterId));
+        }
+        return workers;
+    }
+
+    /**
+     * In case in the future RAS can only use a subset of nodes
+     */
+    private Map<String, RAS_Node> getAvailNodes() {
+        return _nodes;
+    }
+
+    /**
+     * Breadth first traversal of the topology DAG
+     * @param topologies
+     * @param td
+     * @param spouts
+     * @return A partial ordering of components
+     */
+    private Queue<Component> bfs(Topologies topologies, TopologyDetails td, List<Component> spouts) {
+        // Since queue is a interface
+        Queue<Component> ordered__Component_list = new LinkedList<Component>();
+        HashMap<String, Component> visited = new HashMap<>();
+
+        /* start from each spout that is not visited, each does a breadth-first traverse */
+        for (Component spout : spouts) {
+            if (!visited.containsKey(spout.id)) {
+                Queue<Component> queue = new LinkedList<>();
+                queue.offer(spout);
+                while (!queue.isEmpty()) {
+                    Component comp = queue.poll();
+                    visited.put(comp.id, comp);
+                    ordered__Component_list.add(comp);
+                    List<String> neighbors = new ArrayList<>();
+                    neighbors.addAll(comp.children);
+                    neighbors.addAll(comp.parents);
+                    for (String nbID : neighbors) {
+                        if (!visited.containsKey(nbID)) {
+                            Component child = topologies.getAllComponents().get(td.getId()).get(nbID);
+                            queue.offer(child);
+                        }
+                    }
+                }
+            }
+        }
+        return ordered__Component_list;
+    }
+
+    private List<Component> getSpouts(Topologies topologies, TopologyDetails td) {
+        List<Component> spouts = new ArrayList<>();
+        for (Component c : topologies.getAllComponents().get(td.getId())
+                .values()) {
+            if (c.type == Component.ComponentType.SPOUT) {
+                spouts.add(c);
+            }
+        }
+        return spouts;
+    }
+
+    private Integer getLongestPriorityListSize(Map<Integer, List<ExecutorDetails>> priorityToExecutorMap) {
+        Integer mostNum = 0;
+        for (List<ExecutorDetails> execs : priorityToExecutorMap.values()) {
+            Integer numExecs = execs.size();
+            if (mostNum < numExecs) {
+                mostNum = numExecs;
+            }
+        }
+        return mostNum;
+    }
+
+    /**
+     * Get the remaining amount memory that can be assigned to a worker given the set worker max heap size
+     * @param ws
+     * @param td
+     * @param scheduleAssignmentMap
+     * @return The remaining amount of memory
+     */
+    private Double getWorkerScheduledMemoryAvailable(WorkerSlot ws, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
+        Double memScheduleUsed = this.getWorkerScheduledMemoryUse(ws, td, scheduleAssignmentMap);
+        return td.getTopologyWorkerMaxHeapSize() - memScheduleUsed;
+    }
+
+    /**
+     * Get the amount of memory already assigned to a worker
+     * @param ws
+     * @param td
+     * @param scheduleAssignmentMap
+     * @return the amount of memory
+     */
+    private Double getWorkerScheduledMemoryUse(WorkerSlot ws, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
+        Double totalMem = 0.0;
+        Collection<ExecutorDetails> execs = scheduleAssignmentMap.get(ws);
+        if(execs != null) {
+            for(ExecutorDetails exec : execs) {
+                totalMem += td.getTotalMemReqTask(exec);
+            }
+        } 
+        return totalMem;
+    }
+
+    /**
+     * Checks whether we can schedule an Executor exec on the worker slot ws
+     * Only considers memory currently.  May include CPU in the future
+     * @param exec
+     * @param ws
+     * @param td
+     * @param scheduleAssignmentMap
+     * @return a boolean: True denoting the exec can be scheduled on ws and false if it cannot
+     */
+    private boolean checkWorkerConstraints(ExecutorDetails exec, WorkerSlot ws, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
+        boolean retVal = false;
+        if(this.getWorkerScheduledMemoryAvailable(ws, td, scheduleAssignmentMap) >= td.getTotalMemReqTask(exec)) {
+            retVal = true;
+        }
+        return retVal;
+    }
+
+    /**
+     * Get the amount of resources available and total for each node
+     * @return a String with cluster resource info for debug
+     */
+    private String getClusterInfo() {
+        String retVal = "Cluster info:\n";
+        for(Entry<String, List<String>> clusterEntry : _clusterInfo.entrySet()) {
+            String clusterId = clusterEntry.getKey();
+            retVal += "Rack: " + clusterId + "\n";
+            for(String nodeHostname : clusterEntry.getValue()) {
+                RAS_Node node = this.idToNode(this.NodeHostnameToId(nodeHostname));
+                retVal += "-> Node: " + node.getHostname() + " " + node.getId() + "\n";
+                retVal += "--> Avail Resources: {Mem " + node.getAvailableMemoryResources() + ", CPU " + node.getAvailableCpuResources() + "}\n";
+                retVal += "--> Total Resources: {Mem " + node.getTotalMemoryResources() + ", CPU " + node.getTotalCpuResources() + "}\n";
+            }
+        }
+        return retVal;
+    }
+
+    /**
+     * hostname to Id
+     * @param hostname
+     * @return the id of a node
+     */
+    public String NodeHostnameToId(String hostname) {
+        for (RAS_Node n : _nodes.values()) {
+            if (n.getHostname() == null) {
+                continue;
+            }
+            if (n.getHostname().equals(hostname)) {
+                return n.getId();
+            }
+        }
+        LOG.error("Cannot find Node with hostname {}", hostname);
+        return null;
+    }
+
+    /**
+     * Find RAS_Node for specified node id
+     * @param id
+     * @return a RAS_Node object
+     */
+    public RAS_Node idToNode(String id) {
+        if(_nodes.containsKey(id) == false) {
+            LOG.error("Cannot find Node with Id: {}", id);
+            return null;
+        }
+        return _nodes.get(id);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4cd5efa3/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java
new file mode 100644
index 0000000..12e8ff3
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java
@@ -0,0 +1,43 @@
+/**
+ * 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 backtype.storm.scheduler.resource.strategies.scheduling;
+
+import java.util.Collection;
+import java.util.Map;
+
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.ExecutorDetails;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.scheduler.WorkerSlot;
+import backtype.storm.scheduler.resource.RAS_Node;
+import backtype.storm.scheduler.resource.RAS_Nodes;
+import backtype.storm.scheduler.resource.SchedulingResult;
+import backtype.storm.scheduler.resource.User;
+
+/**
+ * An interface to for implementing different scheduling strategies for the resource aware scheduling
+ * In the future stategies will be pluggable
+ */
+public interface IStrategy {
+
+    public void prepare(Topologies topologies, Cluster cluster, Map<String, User> userMap, RAS_Nodes nodes);
+
+    public SchedulingResult schedule(TopologyDetails td);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4cd5efa3/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java b/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
index 52c4ed1..b82a4ec 100644
--- a/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
+++ b/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
@@ -523,6 +523,31 @@ public class ConfigValidation {
         }
     }
 
+    public static class ImplementsClassValidator extends Validator {
+
+        Class classImplements;
+
+        public ImplementsClassValidator(Map<String, Object> params) {
+            this.classImplements = (Class) params.get(ConfigValidationAnnotations.ValidatorParams.IMPLEMENTS_CLASS);
+        }
+
+        @Override
+        public void validateField(String name, Object o) {
+            if(o == null) {
+                return;
+            }
+            SimpleTypeValidator.validateField(name, String.class, o);
+            try {
+                Class objectClass = Class.forName((String) o);
+                if(!this.classImplements.isAssignableFrom(objectClass)) {
+                    throw new IllegalArgumentException("Field " + name + " with value " + o + " does not implement " + this.classImplements.getName());
+                }
+            } catch (ClassNotFoundException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
     /**
      * Methods for validating confs
      */

http://git-wip-us.apache.org/repos/asf/storm/blob/4cd5efa3/storm-core/src/jvm/backtype/storm/validation/ConfigValidationAnnotations.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/validation/ConfigValidationAnnotations.java b/storm-core/src/jvm/backtype/storm/validation/ConfigValidationAnnotations.java
index c47f523..28707e4 100644
--- a/storm-core/src/jvm/backtype/storm/validation/ConfigValidationAnnotations.java
+++ b/storm-core/src/jvm/backtype/storm/validation/ConfigValidationAnnotations.java
@@ -46,6 +46,7 @@ public class ConfigValidationAnnotations {
         static final String VALUE_TYPE = "valueType";
         static final String INCLUDE_ZERO = "includeZero";
         static final String ACCEPTED_VALUES = "acceptedValues";
+        static final String IMPLEMENTS_CLASS = "implementsClass";
     }
 
     /**
@@ -176,6 +177,14 @@ public class ConfigValidationAnnotations {
         boolean includeZero() default false;
     }
 
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isImplementationOfClass {
+        Class validatorClass() default ConfigValidation.ImplementsClassValidator.class;
+
+        Class implementsClass();
+    }
+
     /**
      * Complex/custom type validators
      */


[09/23] storm git commit: first initial implementation

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj b/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
index 4fff405..3f2610c 100644
--- a/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
+++ b/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
@@ -49,6 +49,9 @@
       (into {}
         (for [at (range start end)]
           {(ed at) name})))))
+(def DEFAULT_PRIORITY_STRATEGY "backtype.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy")
+(def DEFAULT_EVICTION_STRATEGY "backtype.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy")
+(def DEFAULT_SCHEDULING_STRATEGY "backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy")
 
 ;; get the super->mem HashMap by counting the eds' mem usage of all topos on each super
 (defn get-super->mem-usage [^Cluster cluster ^Topologies topologies]
@@ -154,7 +157,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
-                     TOPOLOGY-PRIORITY 0}
+                     TOPOLOGY-PRIORITY 0
+                     TOPOLOGY-SCHEDULER-STRATEGY DEFAULT_SCHEDULING_STRATEGY}
                     storm-topology
                     1
                     (mk-ed-map [["wordSpout" 0 1]
@@ -165,7 +169,8 @@
         topologies (Topologies. (to-top-map [topology1]))
         node-map (RAS_Nodes/getAllNodesFrom cluster topologies)
         scheduler (ResourceAwareScheduler.)]
-    (.prepare scheduler {})
+    (.prepare scheduler {RESOURCE-AWARE-SCHEDULER-EVICTION-STRATEGY DEFAULT_EVICTION_STRATEGY
+                         RESOURCE-AWARE-SCHEDULER-PRIORITY-STRATEGY DEFAULT_PRIORITY_STRATEGY})
     (.schedule scheduler topologies cluster)
     (let [assignment (.getAssignmentById cluster "topology1")
           assigned-slots (.getSlots assignment)
@@ -173,7 +178,7 @@
       (is (= 1 (.size assigned-slots)))
       (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
       (is (= 2 (.size executors))))
-    (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))))
+    (is (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap cluster) "topology1")))))
 
 (deftest test-topology-with-multiple-spouts
   (let [builder1 (TopologyBuilder.)  ;; a topology with multiple spouts
@@ -195,7 +200,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
-                     TOPOLOGY-PRIORITY 0}
+                     TOPOLOGY-PRIORITY 0
+                     TOPOLOGY-SCHEDULER-STRATEGY DEFAULT_SCHEDULING_STRATEGY}
                     storm-topology1
                     1
                     (mk-ed-map [["wordSpout1" 0 1]
@@ -216,7 +222,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
-                     TOPOLOGY-PRIORITY 0}
+                     TOPOLOGY-PRIORITY 0
+                     TOPOLOGY-SCHEDULER-STRATEGY DEFAULT_SCHEDULING_STRATEGY}
                     storm-topology2
                     1
                     (mk-ed-map [["wordSpoutX" 0 1]
@@ -227,7 +234,8 @@
                    "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
         topologies (Topologies. (to-top-map [topology1 topology2]))
         scheduler (ResourceAwareScheduler.)]
-    (.prepare scheduler {})
+    (.prepare scheduler {RESOURCE-AWARE-SCHEDULER-EVICTION-STRATEGY DEFAULT_EVICTION_STRATEGY
+                         RESOURCE-AWARE-SCHEDULER-PRIORITY-STRATEGY DEFAULT_PRIORITY_STRATEGY})
     (.schedule scheduler topologies cluster)
     (let [assignment (.getAssignmentById cluster "topology1")
           assigned-slots (.getSlots assignment)
@@ -235,14 +243,14 @@
       (is (= 1 (.size assigned-slots)))
       (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
       (is (= 7 (.size executors))))
-    (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
+    (is (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap cluster) "topology1")))
     (let [assignment (.getAssignmentById cluster "topology2")
           assigned-slots (.getSlots assignment)
           executors (.getExecutors assignment)]
       (is (= 1 (.size assigned-slots)))
       (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
       (is (= 2 (.size executors))))
-    (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))
+    (is (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap cluster) "topology2")))))
 
 (deftest test-topology-set-memory-and-cpu-load
   (let [builder (TopologyBuilder.)
@@ -261,7 +269,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
-                     TOPOLOGY-PRIORITY 0}
+                     TOPOLOGY-PRIORITY 0
+                     TOPOLOGY-SCHEDULER-STRATEGY DEFAULT_SCHEDULING_STRATEGY}
                     storm-topology
                     2
                     (mk-ed-map [["wordSpout" 0 1]
@@ -271,7 +280,8 @@
                    "backtype.storm.testing.AlternateRackDNSToSwitchMapping"})
         topologies (Topologies. (to-top-map [topology2]))
         scheduler (ResourceAwareScheduler.)]
-    (.prepare scheduler {})
+    (.prepare scheduler {RESOURCE-AWARE-SCHEDULER-EVICTION-STRATEGY DEFAULT_EVICTION_STRATEGY
+                         RESOURCE-AWARE-SCHEDULER-PRIORITY-STRATEGY DEFAULT_PRIORITY_STRATEGY})
     (.schedule scheduler topologies cluster)
     (let [assignment (.getAssignmentById cluster "topology2")
           assigned-slots (.getSlots assignment)
@@ -280,7 +290,7 @@
       (is (= 1 (.size assigned-slots)))
       (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
       (is (= 2 (.size executors))))
-    (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))
+    (is (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap cluster) "topology2")))))
 
 (deftest test-resource-limitation
   (let [builder (TopologyBuilder.)
@@ -300,7 +310,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
-                     TOPOLOGY-PRIORITY 0}
+                     TOPOLOGY-PRIORITY 0
+                     TOPOLOGY-SCHEDULER-STRATEGY DEFAULT_SCHEDULING_STRATEGY}
                     storm-topology
                     2 ;; need two workers, each on one node
                     (mk-ed-map [["wordSpout" 0 2]
@@ -310,7 +321,8 @@
                    "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
         topologies (Topologies. (to-top-map [topology1]))
         scheduler (ResourceAwareScheduler.)]
-    (.prepare scheduler {})
+    (.prepare scheduler {RESOURCE-AWARE-SCHEDULER-EVICTION-STRATEGY DEFAULT_EVICTION_STRATEGY
+                         RESOURCE-AWARE-SCHEDULER-PRIORITY-STRATEGY DEFAULT_PRIORITY_STRATEGY})
     (.schedule scheduler topologies cluster)
     (let [assignment (.getAssignmentById cluster "topology1")
           assigned-slots (.getSlots assignment)
@@ -344,7 +356,7 @@
       (is (>= avail used)))
     (doseq [[avail used] cpu-avail->used] ;; for each node, assigned cpu smaller than total
       (is (>= avail used))))
-  (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))))
+  (is (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap cluster) "topology1")))))
 
 (deftest test-scheduling-resilience
   (let [supers (gen-supervisors 2 2)
@@ -358,7 +370,8 @@
                       TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                       TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
                       TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
-                      TOPOLOGY-PRIORITY 0}
+                      TOPOLOGY-PRIORITY 0
+                      TOPOLOGY-SCHEDULER-STRATEGY DEFAULT_SCHEDULING_STRATEGY}
                      storm-topology1
                      3 ;; three workers to hold three executors
                      (mk-ed-map [["spout1" 0 3]]))
@@ -372,7 +385,8 @@
                       TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                       TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
                       TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
-                      TOPOLOGY-PRIORITY 0}
+                      TOPOLOGY-PRIORITY 0
+                      TOPOLOGY-SCHEDULER-STRATEGY DEFAULT_SCHEDULING_STRATEGY}
                      storm-topology2
                      2  ;; two workers, each holds one executor and resides on one node
                      (mk-ed-map [["spout2" 0 2]]))
@@ -383,7 +397,8 @@
                               {STORM-NETWORK-TOPOGRAPHY-PLUGIN
                                "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
             topologies (Topologies. (to-top-map [topology2]))
-            _ (.prepare scheduler {})
+            _ (.prepare scheduler {RESOURCE-AWARE-SCHEDULER-EVICTION-STRATEGY DEFAULT_EVICTION_STRATEGY
+                                   RESOURCE-AWARE-SCHEDULER-PRIORITY-STRATEGY DEFAULT_PRIORITY_STRATEGY})
             _ (.schedule scheduler topologies cluster)
             assignment (.getAssignmentById cluster "topology2")
             failed-worker (first (vec (.getSlots assignment)))  ;; choose a worker to mock as failed
@@ -392,14 +407,15 @@
             _ (doseq [ed failed-eds] (.remove ed->slot ed))  ;; remove executor details assigned to the worker
             copy-old-mapping (HashMap. ed->slot)
             healthy-eds (.keySet copy-old-mapping)
-            _ (.prepare scheduler {})
+            _ (.prepare scheduler {RESOURCE-AWARE-SCHEDULER-EVICTION-STRATEGY DEFAULT_EVICTION_STRATEGY
+                                   RESOURCE-AWARE-SCHEDULER-PRIORITY-STRATEGY DEFAULT_PRIORITY_STRATEGY})
             _ (.schedule scheduler topologies cluster)
             new-assignment (.getAssignmentById cluster "topology2")
             new-ed->slot (.getExecutorToSlot new-assignment)]
         ;; for each executor that was scheduled on healthy workers, their slots should remain unchanged after a new scheduling
         (doseq [ed healthy-eds]
           (is (.equals (.get copy-old-mapping ed) (.get new-ed->slot ed))))
-        (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))
+        (is (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap cluster) "topology2")))))
 
     (testing "When a supervisor fails, RAS does not alter existing assignments"
       (let [existing-assignments {"topology1" (SchedulerAssignmentImpl. "topology1"
@@ -419,7 +435,8 @@
                                   (.getAssignments cluster)
                                   {STORM-NETWORK-TOPOGRAPHY-PLUGIN
                                    "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
-            _ (.prepare scheduler {})
+            _ (.prepare scheduler {RESOURCE-AWARE-SCHEDULER-EVICTION-STRATEGY DEFAULT_EVICTION_STRATEGY
+                                   RESOURCE-AWARE-SCHEDULER-PRIORITY-STRATEGY DEFAULT_PRIORITY_STRATEGY})
             _ (.schedule scheduler topologies new-cluster) ;; the actual schedule for this topo will not run since it is fully assigned
             new-assignment (.getAssignmentById new-cluster "topology1")
             new-ed->slot (.getExecutorToSlot new-assignment)]
@@ -446,33 +463,36 @@
                                   (.getAssignments cluster)
                                   {STORM-NETWORK-TOPOGRAPHY-PLUGIN
                                    "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
-            _ (.prepare scheduler {})
+            _ (.prepare scheduler {RESOURCE-AWARE-SCHEDULER-EVICTION-STRATEGY DEFAULT_EVICTION_STRATEGY
+                                   RESOURCE-AWARE-SCHEDULER-PRIORITY-STRATEGY DEFAULT_PRIORITY_STRATEGY})
             _ (.schedule scheduler topologies new-cluster)
             new-assignment (.getAssignmentById new-cluster "topology1")
             new-ed->slot (.getExecutorToSlot new-assignment)]
         (doseq [ed existing-eds]
           (is (.equals (.get copy-old-mapping ed) (.get new-ed->slot ed))))
-        (is (= "Fully Scheduled" (.get (.getStatusMap new-cluster) "topology1")))))
+        (is (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap new-cluster) "topology1")))))
 
     (testing "Scheduling a new topology does not disturb other assignments unnecessarily"
       (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
                               {STORM-NETWORK-TOPOGRAPHY-PLUGIN
                                "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
             topologies (Topologies. (to-top-map [topology1]))
-            _ (.prepare scheduler {})
+            _ (.prepare scheduler {RESOURCE-AWARE-SCHEDULER-EVICTION-STRATEGY DEFAULT_EVICTION_STRATEGY
+                                   RESOURCE-AWARE-SCHEDULER-PRIORITY-STRATEGY DEFAULT_PRIORITY_STRATEGY})
             _ (.schedule scheduler topologies cluster)
             assignment (.getAssignmentById cluster "topology1")
             ed->slot (.getExecutorToSlot assignment)
             copy-old-mapping (HashMap. ed->slot)
             new-topologies (Topologies. (to-top-map [topology1 topology2]))  ;; a second topology joins
-            _ (.prepare scheduler {})
+            _ (.prepare scheduler {RESOURCE-AWARE-SCHEDULER-EVICTION-STRATEGY DEFAULT_EVICTION_STRATEGY
+                                   RESOURCE-AWARE-SCHEDULER-PRIORITY-STRATEGY DEFAULT_PRIORITY_STRATEGY})
             _ (.schedule scheduler new-topologies cluster)
             new-assignment (.getAssignmentById cluster "topology1")
             new-ed->slot (.getExecutorToSlot new-assignment)]
         (doseq [ed (.keySet copy-old-mapping)]
           (is (.equals (.get copy-old-mapping ed) (.get new-ed->slot ed))))  ;; the assignment for topo1 should not change
-        (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
-        (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))))
+        (is (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap cluster) "topology1")))
+        (is (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap cluster) "topology2")))))))
 
 ;; Automated tests for heterogeneous cluster
 (deftest test-heterogeneous-cluster
@@ -497,7 +517,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
-                     TOPOLOGY-PRIORITY 0}
+                     TOPOLOGY-PRIORITY 0
+                     TOPOLOGY-SCHEDULER-STRATEGY DEFAULT_SCHEDULING_STRATEGY}
                     storm-topology1
                     1
                     (mk-ed-map [["spout1" 0 1]]))
@@ -513,7 +534,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
-                     TOPOLOGY-PRIORITY 0}
+                     TOPOLOGY-PRIORITY 0
+                     TOPOLOGY-SCHEDULER-STRATEGY DEFAULT_SCHEDULING_STRATEGY}
                     storm-topology2
                     2
                     (mk-ed-map [["spout2" 0 4]]))
@@ -529,7 +551,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
-                     TOPOLOGY-PRIORITY 0}
+                     TOPOLOGY-PRIORITY 0
+                     TOPOLOGY-SCHEDULER-STRATEGY DEFAULT_SCHEDULING_STRATEGY}
                     storm-topology3
                     2
                     (mk-ed-map [["spout3" 0 4]]))
@@ -545,7 +568,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
-                     TOPOLOGY-PRIORITY 0}
+                     TOPOLOGY-PRIORITY 0
+                     TOPOLOGY-SCHEDULER-STRATEGY DEFAULT_SCHEDULING_STRATEGY}
                     storm-topology4
                     2
                     (mk-ed-map [["spout4" 0 12]]))
@@ -561,7 +585,8 @@
                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0
-                     TOPOLOGY-PRIORITY 0}
+                     TOPOLOGY-PRIORITY 0
+                     TOPOLOGY-SCHEDULER-STRATEGY DEFAULT_SCHEDULING_STRATEGY}
                     storm-topology5
                     2
                     (mk-ed-map [["spout5" 0 40]]))
@@ -574,13 +599,14 @@
                                "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
             topologies (Topologies. (to-top-map [topology1 topology2 topology3]))
             scheduler (ResourceAwareScheduler.)
-            _ (.prepare scheduler {})
+            _ (.prepare scheduler {RESOURCE-AWARE-SCHEDULER-EVICTION-STRATEGY DEFAULT_EVICTION_STRATEGY
+                                   RESOURCE-AWARE-SCHEDULER-PRIORITY-STRATEGY DEFAULT_PRIORITY_STRATEGY})
             _ (.schedule scheduler topologies cluster)
             super->mem-usage (get-super->mem-usage cluster topologies)
             super->cpu-usage (get-super->cpu-usage cluster topologies)]
-        (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
-        (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))
-        (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology3")))
+        (is (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap cluster) "topology1")))
+        (is (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap cluster) "topology2")))
+        (is (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap cluster) "topology3")))
         (doseq [super (.values supers)]
           (let [mem-avail (.getTotalMemory super)
                 mem-used (.get super->mem-usage super)
@@ -595,11 +621,12 @@
                                "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
             topologies (Topologies. (to-top-map [topology1 topology2 topology3]))
             scheduler (ResourceAwareScheduler.)
-            _ (.prepare scheduler {})
+            _ (.prepare scheduler {RESOURCE-AWARE-SCHEDULER-EVICTION-STRATEGY DEFAULT_EVICTION_STRATEGY
+                                   RESOURCE-AWARE-SCHEDULER-PRIORITY-STRATEGY DEFAULT_PRIORITY_STRATEGY})
             _ (.schedule scheduler topologies cluster)
-                scheduled-topos (if (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")) 1 0)
-                scheduled-topos (+ scheduled-topos (if (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")) 1 0))
-                scheduled-topos (+ scheduled-topos (if (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology4")) 1 0))]
+                scheduled-topos (if (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap cluster) "topology1")) 1 0)
+                scheduled-topos (+ scheduled-topos (if (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap cluster) "topology2")) 1 0))
+                scheduled-topos (+ scheduled-topos (if (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap cluster) "topology4")) 1 0))]
             (is (= scheduled-topos 2)))) ;; only 2 topos will get (fully) scheduled
 
     (testing "Launch topo5 only, both mem and cpu should be exactly used up"
@@ -608,11 +635,12 @@
                                "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
             topologies (Topologies. (to-top-map [topology5]))
             scheduler (ResourceAwareScheduler.)
-            _ (.prepare scheduler {})
+            _ (.prepare scheduler {RESOURCE-AWARE-SCHEDULER-EVICTION-STRATEGY DEFAULT_EVICTION_STRATEGY
+                                   RESOURCE-AWARE-SCHEDULER-PRIORITY-STRATEGY DEFAULT_PRIORITY_STRATEGY})
             _ (.schedule scheduler topologies cluster)
             super->mem-usage (get-super->mem-usage cluster topologies)
             super->cpu-usage (get-super->cpu-usage cluster topologies)]
-        (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology5")))
+        (is (= "Running - Fully Scheduled by DefaultResourceAwareStrategy" (.get (.getStatusMap cluster) "topology5")))
         (doseq [super (.values supers)]
           (let [mem-avail (.getTotalMemory super)
                 mem-used (.get super->mem-usage super)
@@ -638,14 +666,16 @@
                        TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                        TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
                        TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 128.0
-                       TOPOLOGY-PRIORITY 0}
+                       TOPOLOGY-PRIORITY 0
+                       TOPOLOGY-SCHEDULER-STRATEGY DEFAULT_SCHEDULING_STRATEGY}
                       storm-topology1
                       1
                       (mk-ed-map [["spout1" 0 4]]))
           topologies (Topologies. (to-top-map [topology1]))]
-      (.prepare scheduler {"userA" {"cpu" 2000.0 "memory" 400.0}})
+      (.prepare scheduler {RESOURCE-AWARE-SCHEDULER-EVICTION-STRATEGY DEFAULT_EVICTION_STRATEGY
+                           RESOURCE-AWARE-SCHEDULER-PRIORITY-STRATEGY DEFAULT_PRIORITY_STRATEGY})
       (.schedule scheduler topologies cluster)
-      (is (= (.get (.getStatusMap cluster) "topology1") "Fully Scheduled"))
+      (is (= (.get (.getStatusMap cluster) "topology1") "Running - Fully Scheduled by DefaultResourceAwareStrategy"))
       (is (= (.getAssignedNumWorkers cluster topology1) 4)))
     (testing "test when no more workers are available due to topology worker max heap size limit but there is memory is still available")
     (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
@@ -662,18 +692,20 @@
                        TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                        TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
                        TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 128.0
-                       TOPOLOGY-PRIORITY 0}
+                       TOPOLOGY-PRIORITY 0
+                       TOPOLOGY-SCHEDULER-STRATEGY DEFAULT_SCHEDULING_STRATEGY}
                       storm-topology1
                       1
                       (mk-ed-map [["spout1" 0 5]]))
           topologies (Topologies. (to-top-map [topology1]))]
-      (.prepare scheduler {})
+      (.prepare scheduler {RESOURCE-AWARE-SCHEDULER-EVICTION-STRATEGY DEFAULT_EVICTION_STRATEGY
+                           RESOURCE-AWARE-SCHEDULER-PRIORITY-STRATEGY DEFAULT_PRIORITY_STRATEGY})
       (.schedule scheduler topologies cluster)
       ;;spout1 is going to contain 5 executors that needs scheduling. Each of those executors has a memory requirement of 128.0 MB
       ;;The cluster contains 4 free WorkerSlots. For this topolology each worker is limited to a max heap size of 128.0
       ;;Thus, one executor not going to be able to get scheduled thus failing the scheduling of this topology and no executors of this topology will be scheduleded
       (is (= (.size (.getUnassignedExecutors cluster topology1)) 5))
-      (is (= (.get (.getStatusMap cluster) "topology1")  "Not all executors successfully scheduled: [[1, 1]]")))
+      (is (= (.get (.getStatusMap cluster) "topology1")  "Not enough resources to schedule - 0/5 executors scheduled")))
 
     (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
                              {STORM-NETWORK-TOPOGRAPHY-PLUGIN
@@ -688,7 +720,8 @@
                  TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
                  TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
                  TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 128.0
-                 TOPOLOGY-PRIORITY 0}
+                 TOPOLOGY-PRIORITY 0
+                 TOPOLOGY-SCHEDULER-STRATEGY DEFAULT_SCHEDULING_STRATEGY}
           topology1 (TopologyDetails. "topology1"
                       conf
                       storm-topology1

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/test/jvm/backtype/storm/scheduler/resource/Experiment.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/Experiment.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/Experiment.java
deleted file mode 100644
index 3ff0af1..0000000
--- a/storm-core/test/jvm/backtype/storm/scheduler/resource/Experiment.java
+++ /dev/null
@@ -1,222 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <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 backtype.storm.scheduler.resource;
-
-import backtype.storm.Config;
-import backtype.storm.scheduler.Cluster;
-import backtype.storm.scheduler.INimbus;
-import backtype.storm.scheduler.SchedulerAssignmentImpl;
-import backtype.storm.scheduler.SupervisorDetails;
-import backtype.storm.scheduler.Topologies;
-import backtype.storm.scheduler.TopologyDetails;
-import backtype.storm.utils.Time;
-import backtype.storm.utils.Utils;
-import org.junit.Test;
-import org.junit.Assert;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Created by jerrypeng on 11/11/15.
- */
-public class Experiment {
-
-    private static final Logger LOG = LoggerFactory.getLogger(Experiment.class);
-
-    /**
-     * Eviction order:
-     * topo-3: since user bobby don't have any resource guarantees and topo-3 is the lowest priority for user bobby
-     * topo-2: since user bobby don't have any resource guarantees and topo-2 is the next lowest priority for user bobby
-     * topo-5: since user derek has exceeded his resource guarantee while user jerry has not.  topo-5 and topo-4 has the same priority
-     * but topo-4 was submitted earlier thus we choose that one to evict
-     */
-    @Test
-    public void TestEvictMultipleTopologiesFromMultipleUsersInCorrectOrder() {
-        INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
-        Map<String, Number> resourceMap = new HashMap<String, Number>();
-        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 100.0);
-        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1000.0);
-        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(4, 4, resourceMap);
-        Config config = new Config();
-        config.putAll(Utils.readDefaultConfig());
-        config.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 100.0);
-        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 500);
-        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 500);
-        Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
-        resourceUserPool.put("jerry", new HashMap<String, Number>());
-        resourceUserPool.get("jerry").put("cpu", 300.0);
-        resourceUserPool.get("jerry").put("memory", 3000.0);
-
-        resourceUserPool.put("derek", new HashMap<String, Number>());
-        resourceUserPool.get("derek").put("cpu", 100.0);
-        resourceUserPool.get("derek").put("memory", 1000.0);
-
-        config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
-        Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), config);
-
-        config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
-
-        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
-        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
-        TopologyDetails topo7 = TestUtilsForResourceAwareScheduler.getTopology("topo-7", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
-
-        config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
-
-        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
-        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
-
-        config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
-
-        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
-        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 15, 30);
-
-        Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
-        topoMap.put(topo2.getId(), topo2);
-        topoMap.put(topo3.getId(), topo3);
-        topoMap.put(topo4.getId(), topo4);
-        topoMap.put(topo5.getId(), topo5);
-
-        Topologies topologies = new Topologies(topoMap);
-
-        ResourceAwareScheduler rs = new ResourceAwareScheduler();
-
-        rs.prepare(config);
-        rs.schedule(topologies, cluster);
-
-        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
-            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
-        }
-        Assert.assertEquals("# of running topologies", 2, rs.getUser("derek").getTopologiesRunning().size());
-        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
-        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
-        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
-
-        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
-            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
-        }
-        Assert.assertEquals("# of running topologies", 2, rs.getUser("bobby").getTopologiesRunning().size());
-        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
-        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
-        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("bobby").getTopologiesAttempted().size());
-
-        //user jerry submits another topology
-        topoMap.put(topo1.getId(), topo1);
-        topologies = new Topologies(topoMap);
-        rs.schedule(topologies, cluster);
-
-        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
-            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
-        }
-        Assert.assertEquals("# of running topologies", 1, rs.getUser("jerry").getTopologiesRunning().size());
-        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
-        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
-        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
-
-        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
-            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
-        }
-        Assert.assertEquals("# of running topologies", 2, rs.getUser("derek").getTopologiesRunning().size());
-        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
-        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
-        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
-
-        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesAttempted()) {
-            Assert.assertFalse("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
-        }
-        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
-            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
-        }
-        Assert.assertEquals("# of attempted topologies", 1, rs.getUser("bobby").getTopologiesAttempted().size());
-        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
-        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
-        Assert.assertEquals("# of running topologies", 1, rs.getUser("bobby").getTopologiesRunning().size());
-        Assert.assertEquals("correct topology to evict", rs.getUser("bobby").getTopologiesAttempted().iterator().next().getName(), "topo-3");
-
-        topoMap.put(topo6.getId(), topo6);
-        topologies = new Topologies(topoMap);
-        rs.schedule(topologies, cluster);
-
-        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
-            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
-        }
-        Assert.assertEquals("# of running topologies", 2, rs.getUser("jerry").getTopologiesRunning().size());
-        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
-        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
-        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
-
-        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
-            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
-        }
-        Assert.assertEquals("# of running topologies", 2, rs.getUser("derek").getTopologiesRunning().size());
-        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
-        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
-        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
-
-        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesAttempted()) {
-            Assert.assertFalse(TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
-        }
-        Assert.assertEquals("# of attempted topologies", 2, rs.getUser("bobby").getTopologiesAttempted().size());
-        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
-        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
-        Assert.assertEquals("# of running topologies", 0, rs.getUser("bobby").getTopologiesRunning().size());
-
-        Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-2", rs.getUser("bobby").getTopologiesAttempted()) != null);
-        Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-3", rs.getUser("bobby").getTopologiesAttempted()) != null);
-
-        topoMap.put(topo7.getId(), topo7);
-        topologies = new Topologies(topoMap);
-        rs.schedule(topologies, cluster);
-
-        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
-            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
-        }
-        Assert.assertEquals("# of running topologies", 3, rs.getUser("jerry").getTopologiesRunning().size());
-        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
-        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
-        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
-
-        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
-            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
-        }
-        for (TopologyDetails topo : rs.getUser("derek").getTopologiesAttempted()) {
-            Assert.assertFalse("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
-        }
-        Assert.assertEquals("# of running topologies", 1, rs.getUser("derek").getTopologiesRunning().size());
-        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
-        Assert.assertEquals("# of attempted topologies", 1, rs.getUser("derek").getTopologiesAttempted().size());
-        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
-        Assert.assertEquals("correct topology to evict", rs.getUser("derek").getTopologiesAttempted().iterator().next().getName(), "topo-4");
-
-        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesAttempted()) {
-            Assert.assertFalse("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
-        }
-        Assert.assertEquals("# of attempted topologies", 2, rs.getUser("bobby").getTopologiesAttempted().size());
-        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
-        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
-        Assert.assertEquals("# of running topologies", 0, rs.getUser("bobby").getTopologiesRunning().size());
-
-        Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-2", rs.getUser("bobby").getTopologiesAttempted()) != null);
-        Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-3", rs.getUser("bobby").getTopologiesAttempted()) != null);
-    }
-
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
index b41b039..ffdf460 100644
--- a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
+++ b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
@@ -20,8 +20,10 @@ package backtype.storm.scheduler.resource;
 
 import backtype.storm.Config;
 import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.ExecutorDetails;
 import backtype.storm.scheduler.INimbus;
 import backtype.storm.scheduler.IScheduler;
+import backtype.storm.scheduler.SchedulerAssignment;
 import backtype.storm.scheduler.SchedulerAssignmentImpl;
 import backtype.storm.scheduler.SupervisorDetails;
 import backtype.storm.scheduler.Topologies;
@@ -66,6 +68,9 @@ public class TestResourceAwareScheduler {
         Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(20, 4, resourceMap);
         Config config = new Config();
         config.putAll(Utils.readDefaultConfig());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY, backtype.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy.class.getName());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, backtype.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy.class.getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class.getName());
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, TOPOLOGY_SUBMITTER);
 
@@ -175,6 +180,9 @@ public class TestResourceAwareScheduler {
         Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(20, 4, resourceMap);
         Config config = new Config();
         config.putAll(Utils.readDefaultConfig());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY, backtype.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy.class.getName());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, backtype.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy.class.getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class.getName());
         Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
         resourceUserPool.put("jerry", new HashMap<String, Number>());
         resourceUserPool.get("jerry").put("cpu", 1000);
@@ -240,7 +248,7 @@ public class TestResourceAwareScheduler {
         rs.schedule(topologies, cluster);
 
         for (TopologyDetails topo : topoMap.values()) {
-            Assert.assertEquals(cluster.getStatusMap().get(topo.getId()), "Fully Scheduled");
+            Assert.assertTrue(TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
 
         for (User user : rs.getUserMap().values()) {
@@ -258,6 +266,9 @@ public class TestResourceAwareScheduler {
         Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(1, 4, resourceMap);
         Config config = new Config();
         config.putAll(Utils.readDefaultConfig());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY, backtype.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy.class.getName());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, backtype.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy.class.getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class.getName());
         Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
         resourceUserPool.put("jerry", new HashMap<String, Number>());
         resourceUserPool.get("jerry").put("cpu", 1000);
@@ -316,6 +327,9 @@ public class TestResourceAwareScheduler {
         Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(4, 4, resourceMap);
         Config config = new Config();
         config.putAll(Utils.readDefaultConfig());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY, backtype.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy.class.getName());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, backtype.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy.class.getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class.getName());
         config.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 100.0);
         config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 500);
         config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 500);
@@ -432,6 +446,9 @@ public class TestResourceAwareScheduler {
         Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(4, 4, resourceMap);
         Config config = new Config();
         config.putAll(Utils.readDefaultConfig());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY, backtype.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy.class.getName());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, backtype.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy.class.getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class.getName());
         config.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 100.0);
         config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 500);
         config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 500);
@@ -539,6 +556,9 @@ public class TestResourceAwareScheduler {
         Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(4, 4, resourceMap);
         Config config = new Config();
         config.putAll(Utils.readDefaultConfig());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY, backtype.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy.class.getName());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, backtype.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy.class.getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class.getName());
         config.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 100.0);
         config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 500);
         config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 500);
@@ -653,7 +673,7 @@ public class TestResourceAwareScheduler {
         Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
 
         for (TopologyDetails topo : rs.getUser("bobby").getTopologiesAttempted()) {
-            Assert.assertFalse(TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+            Assert.assertFalse("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
         Assert.assertEquals("# of attempted topologies", 2, rs.getUser("bobby").getTopologiesAttempted().size());
         Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
@@ -698,4 +718,459 @@ public class TestResourceAwareScheduler {
         Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-2", rs.getUser("bobby").getTopologiesAttempted()) != null);
         Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-3", rs.getUser("bobby").getTopologiesAttempted()) != null);
     }
+
+    /**
+     * If topologies from other users cannot be evicted to make space
+     * check if there is a topology with lower priority that can be evicted from the current user
+     */
+    @Test
+    public void TestEvictTopologyFromItself() {
+        INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
+        Map<String, Number> resourceMap = new HashMap<String, Number>();
+        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 100.0);
+        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1000.0);
+        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(4, 4, resourceMap);
+        Config config = new Config();
+        config.putAll(Utils.readDefaultConfig());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY, backtype.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy.class.getName());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, backtype.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy.class.getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class.getName());
+        config.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 100.0);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 500);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 500);
+        Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
+        resourceUserPool.put("jerry", new HashMap<String, Number>());
+        resourceUserPool.get("jerry").put("cpu", 200.0);
+        resourceUserPool.get("jerry").put("memory", 2000.0);
+
+        resourceUserPool.put("bobby", new HashMap<String, Number>());
+        resourceUserPool.get("bobby").put("cpu", 100.0);
+        resourceUserPool.get("bobby").put("memory", 1000.0);
+
+
+        resourceUserPool.put("derek", new HashMap<String, Number>());
+        resourceUserPool.get("derek").put("cpu", 100.0);
+        resourceUserPool.get("derek").put("memory", 1000.0);
+
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
+        Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), config);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
+
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
+
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
+
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
+
+        Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
+        topoMap.put(topo1.getId(), topo1);
+        topoMap.put(topo2.getId(), topo2);
+        topoMap.put(topo5.getId(), topo5);
+        topoMap.put(topo6.getId(), topo6);
+
+        Topologies topologies = new Topologies(topoMap);
+
+        ResourceAwareScheduler rs = new ResourceAwareScheduler();
+
+        rs.prepare(config);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("bobby").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("bobby").getTopologiesAttempted().size());
+
+        //user jerry submits another topology into a full cluster
+        // topo3 should not be able to scheduled
+        topoMap.put(topo3.getId(), topo3);
+        topologies = new Topologies(topoMap);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 1, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+        //make sure that topo-3 didn't get scheduled.
+        Assert.assertEquals("correct topology in attempted queue", rs.getUser("jerry").getTopologiesAttempted().iterator().next().getName(), "topo-3");
+
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("bobby").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("bobby").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+
+        //user jerry submits another topology but this one should be scheduled since it has higher priority than than the
+        //rest of jerry's running topologies
+        topoMap.put(topo4.getId(), topo4);
+        topologies = new Topologies(topoMap);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 2, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+        Assert.assertTrue("correct topology in attempted queue", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-3", rs.getUser("jerry").getTopologiesAttempted()) != null);
+        //Either topo-1 or topo-2 should have gotten evicted
+        Assert.assertTrue("correct topology in attempted queue", ((TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-1", rs.getUser("jerry").getTopologiesAttempted())) != null)
+                || (TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-2", rs.getUser("jerry").getTopologiesAttempted()) != null));
+        //assert that topo-4 got scheduled
+        Assert.assertTrue("correct topology in running queue", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-4", rs.getUser("jerry").getTopologiesRunning()) != null);
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("bobby").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("bobby").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+    }
+
+    /**
+     * If topologies from other users cannot be evicted to make space
+     * check if there is a topology with lower priority that can be evicted from the current user
+     */
+    @Test
+    public void TestOverGuaranteeEviction() {
+        INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
+        Map<String, Number> resourceMap = new HashMap<String, Number>();
+        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 100.0);
+        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1000.0);
+        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(4, 4, resourceMap);
+        Config config = new Config();
+        config.putAll(Utils.readDefaultConfig());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY, backtype.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy.class.getName());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, backtype.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy.class.getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class.getName());
+        config.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 100.0);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 500);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 500);
+        Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
+        resourceUserPool.put("jerry", new HashMap<String, Number>());
+        resourceUserPool.get("jerry").put("cpu", 70.0);
+        resourceUserPool.get("jerry").put("memory", 700.0);
+
+        resourceUserPool.put("bobby", new HashMap<String, Number>());
+        resourceUserPool.get("bobby").put("cpu", 100.0);
+        resourceUserPool.get("bobby").put("memory", 1000.0);
+
+
+        resourceUserPool.put("derek", new HashMap<String, Number>());
+        resourceUserPool.get("derek").put("cpu", 25.0);
+        resourceUserPool.get("derek").put("memory", 250.0);
+
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
+        Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), config);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
+
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
+
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
+
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+
+        Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
+        topoMap.put(topo1.getId(), topo1);
+        topoMap.put(topo3.getId(), topo3);
+        topoMap.put(topo4.getId(), topo4);
+        topoMap.put(topo5.getId(), topo5);
+
+        Topologies topologies = new Topologies(topoMap);
+
+        ResourceAwareScheduler rs = new ResourceAwareScheduler();
+
+        rs.prepare(config);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("bobby").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("bobby").getTopologiesAttempted().size());
+
+        //user derek submits another topology into a full cluster
+        // topo6 should not be able to scheduled
+        topoMap.put(topo6.getId(), topo6);
+        topologies = new Topologies(topoMap);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 1, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+        //topo5 will be evicted since topo6 has higher priority
+        Assert.assertEquals("correct topology in attempted queue", "topo-5", rs.getUser("derek").getTopologiesAttempted().iterator().next().getName());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("bobby").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("bobby").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+
+        //user jerry submits topo2
+        topoMap.put(topo2.getId(), topo2);
+        topologies = new Topologies(topoMap);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 0, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 2, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+        Assert.assertEquals("correct topology in attempted queue", "topo-6", rs.getUser("derek").getTopologiesAttempted().iterator().next().getName());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("bobby").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("bobby").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+    }
+
+    /**
+     * Test correct behaviour when a supervisor dies.  Check if the scheduler handles it correctly and evicts the correct
+     * topology when rescheduling the executors from the died supervisor
+     */
+    @Test
+    public void TestFaultTolerance() {
+        INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
+        Map<String, Number> resourceMap = new HashMap<String, Number>();
+        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 100.0);
+        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1000.0);
+        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(6, 4, resourceMap);
+        Config config = new Config();
+        config.putAll(Utils.readDefaultConfig());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY, backtype.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy.class.getName());
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, backtype.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy.class.getName());
+        config.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class.getName());
+        config.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 100.0);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 500);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 500);
+        Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
+        resourceUserPool.put("jerry", new HashMap<String, Number>());
+        resourceUserPool.get("jerry").put("cpu", 50.0);
+        resourceUserPool.get("jerry").put("memory", 500.0);
+
+        resourceUserPool.put("bobby", new HashMap<String, Number>());
+        resourceUserPool.get("bobby").put("cpu", 200.0);
+        resourceUserPool.get("bobby").put("memory", 2000.0);
+
+
+        resourceUserPool.put("derek", new HashMap<String, Number>());
+        resourceUserPool.get("derek").put("cpu", 100.0);
+        resourceUserPool.get("derek").put("memory", 1000.0);
+
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
+        Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), config);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
+
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
+
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
+
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+
+        Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
+        topoMap.put(topo1.getId(), topo1);
+        topoMap.put(topo2.getId(), topo2);
+        topoMap.put(topo3.getId(), topo3);
+        topoMap.put(topo4.getId(), topo4);
+        topoMap.put(topo5.getId(), topo5);
+        topoMap.put(topo6.getId(), topo6);
+
+
+        Topologies topologies = new Topologies(topoMap);
+
+        ResourceAwareScheduler rs = new ResourceAwareScheduler();
+
+        rs.prepare(config);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("bobby").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("bobby").getTopologiesAttempted().size());
+
+        //fail supervisor
+        SupervisorDetails supFailed = cluster.getSupervisors().values().iterator().next();
+        LOG.info("/***** failing supervisor: {} ****/", supFailed.getHost());
+        supMap.remove(supFailed.getId());
+        Map<String, SchedulerAssignmentImpl> newAssignments = new HashMap<String, SchedulerAssignmentImpl>();
+        for (Map.Entry<String, SchedulerAssignment> topoToAssignment : cluster.getAssignments().entrySet()) {
+            String topoId = topoToAssignment.getKey();
+            SchedulerAssignment assignment = topoToAssignment.getValue();
+            Map<ExecutorDetails, WorkerSlot> executorToSlots = new HashMap<ExecutorDetails, WorkerSlot>();
+            for (Map.Entry<ExecutorDetails, WorkerSlot> execToWorker : assignment.getExecutorToSlot().entrySet()) {
+                ExecutorDetails exec = execToWorker.getKey();
+                WorkerSlot ws = execToWorker.getValue();
+                if (!ws.getNodeId().equals(supFailed.getId())) {
+                    executorToSlots.put(exec, ws);
+                }
+            }
+            newAssignments.put(topoId, new SchedulerAssignmentImpl(topoId, executorToSlots));
+        }
+        Map<String, String> statusMap = cluster.getStatusMap();
+        cluster = new Cluster(iNimbus, supMap, newAssignments, config);
+        cluster.setStatusMap(statusMap);
+
+        rs.schedule(topologies, cluster);
+
+        //Supervisor failed contains a executor from topo-6 of user derek.  Should evict a topology from user jerry since user will be above resource guarantee more so than user derek
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 1, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("bobby").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("bobby").getTopologiesAttempted().size());
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUser.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUser.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUser.java
index e9bf039..24ff980 100644
--- a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUser.java
+++ b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUser.java
@@ -104,8 +104,8 @@ public class TestUser {
         Assert.assertEquals("check cpu resource guarantee", cpuGuarantee, user1.getCPUResourceGuaranteed(), 0.001);
         Assert.assertEquals("check memory resource guarantee", memoryGuarantee, user1.getMemoryResourceGuaranteed(), 0.001);
 
-        Assert.assertEquals("check cpu resource pool utilization", ((100.0 * 3.0) / cpuGuarantee), user1.getCPUResourcePoolUtilization().doubleValue(), 0.001);
-        Assert.assertEquals("check memory resource pool utilization", ((200.0 + 200.0) * 3.0) / memoryGuarantee, user1.getMemoryResourcePoolUtilization().doubleValue(), 0.001);
+        Assert.assertEquals("check cpu resource pool utilization", ((100.0 * 3.0) / cpuGuarantee), user1.getCPUResourcePoolUtilization(), 0.001);
+        Assert.assertEquals("check memory resource pool utilization", ((200.0 + 200.0) * 3.0) / memoryGuarantee, user1.getMemoryResourcePoolUtilization(), 0.001);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3e832205/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
index 7721300..2098f0c 100644
--- a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
+++ b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
@@ -6,9 +6,9 @@
  * 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.
@@ -97,31 +97,31 @@ public class TestUtilsForResourceAwareScheduler {
 
     public static Map<String, SupervisorDetails> genSupervisors(int numSup, int numPorts, Map resourceMap) {
         Map<String, SupervisorDetails> retList = new HashMap<String, SupervisorDetails>();
-        for(int i=0; i<numSup; i++) {
+        for (int i = 0; i < numSup; i++) {
             List<Number> ports = new LinkedList<Number>();
-            for(int j = 0; j<numPorts; j++) {
+            for (int j = 0; j < numPorts; j++) {
                 ports.add(j);
             }
-            SupervisorDetails sup = new SupervisorDetails("sup-"+i, "host-"+i, null, ports, resourceMap);
+            SupervisorDetails sup = new SupervisorDetails("sup-" + i, "host-" + i, null, ports, resourceMap);
             retList.put(sup.getId(), sup);
         }
         return retList;
     }
 
     public static Map<ExecutorDetails, String> genExecsAndComps(StormTopology topology, int spoutParallelism, int boltParallelism) {
-        Map<ExecutorDetails, String> retMap = new HashMap<ExecutorDetails, String> ();
-        int startTask=0;
-        int endTask=1;
-        for(Map.Entry<String, SpoutSpec> entry : topology.get_spouts().entrySet()) {
-            for(int i=0; i<spoutParallelism; i++) {
+        Map<ExecutorDetails, String> retMap = new HashMap<ExecutorDetails, String>();
+        int startTask = 0;
+        int endTask = 1;
+        for (Map.Entry<String, SpoutSpec> entry : topology.get_spouts().entrySet()) {
+            for (int i = 0; i < spoutParallelism; i++) {
                 retMap.put(new ExecutorDetails(startTask, endTask), entry.getKey());
                 startTask++;
                 endTask++;
             }
         }
 
-        for(Map.Entry<String, Bolt> entry : topology.get_bolts().entrySet()) {
-            for(int i=0; i<boltParallelism; i++) {
+        for (Map.Entry<String, Bolt> entry : topology.get_bolts().entrySet()) {
+            for (int i = 0; i < boltParallelism; i++) {
                 retMap.put(new ExecutorDetails(startTask, endTask), entry.getKey());
                 startTask++;
                 endTask++;
@@ -131,14 +131,14 @@ public class TestUtilsForResourceAwareScheduler {
     }
 
     public static TopologyDetails getTopology(String name, Map config, int numSpout, int numBolt,
-                                       int spoutParallelism, int boltParallelism, int launchTime, int priority) {
+                                              int spoutParallelism, int boltParallelism, int launchTime, int priority) {
 
         Config conf = new Config();
         conf.putAll(config);
         conf.put(Config.TOPOLOGY_PRIORITY, priority);
         conf.put(Config.TOPOLOGY_NAME, name);
         conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE);
-        StormTopology topology = buildTopology(numSpout,numBolt, spoutParallelism, boltParallelism);
+        StormTopology topology = buildTopology(numSpout, numBolt, spoutParallelism, boltParallelism);
         TopologyDetails topo = new TopologyDetails(name + "-" + launchTime, conf, topology,
                 0,
                 genExecsAndComps(topology, spoutParallelism, boltParallelism), launchTime);
@@ -258,7 +258,7 @@ public class TestUtilsForResourceAwareScheduler {
 
         @Override
         public String getHostName(Map<String, SupervisorDetails> existingSupervisors, String nodeId) {
-            if(existingSupervisors.containsKey(nodeId)) {
+            if (existingSupervisors.containsKey(nodeId)) {
                 return existingSupervisors.get(nodeId).getHost();
             }
             return null;
@@ -270,10 +270,10 @@ public class TestUtilsForResourceAwareScheduler {
         }
     }
 
-    private static boolean isContain(String source, String subItem){
-        String pattern = "\\b"+subItem+"\\b";
-        Pattern p=Pattern.compile(pattern, Pattern.CASE_INSENSITIVE);
-        Matcher m=p.matcher(source);
+    private static boolean isContain(String source, String subItem) {
+        String pattern = "\\b" + subItem + "\\b";
+        Pattern p = Pattern.compile(pattern, Pattern.CASE_INSENSITIVE);
+        Matcher m = p.matcher(source);
         return m.find();
     }
 
@@ -283,8 +283,8 @@ public class TestUtilsForResourceAwareScheduler {
 
     public static TopologyDetails findTopologyInSetFromName(String topoName, Set<TopologyDetails> set) {
         TopologyDetails ret = null;
-        for(TopologyDetails entry : set) {
-            if(entry.getName().equals(topoName)) {
+        for (TopologyDetails entry : set) {
+            if (entry.getName().equals(topoName)) {
                 ret = entry;
             }
         }


[04/23] storm git commit: adding configs and validation

Posted by da...@apache.org.
adding configs and validation


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/97b2c9a7
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/97b2c9a7
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/97b2c9a7

Branch: refs/heads/master
Commit: 97b2c9a7e75c0913472311b0be3d9d042b95e21f
Parents: 0d34abf
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Thu Nov 12 09:46:05 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Fri Dec 4 13:06:45 2015 -0600

----------------------------------------------------------------------
 storm-core/src/jvm/backtype/storm/Config.java   | 22 ++++++++++++++++++++
 .../storm/scheduler/TopologyDetails.java        |  1 +
 2 files changed, 23 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/97b2c9a7/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index ab4d5f6..d9c8815 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -1847,6 +1847,13 @@ public class Config extends HashMap<String, Object> {
     public static final String TOPOLOGY_LOGGING_SENSITIVITY="topology.logging.sensitivity";
 
     /**
+     * Sets the priority for a topology
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_PRIORITY = "topology.priority";
+
+    /**
      * The root directory in ZooKeeper for metadata about TransactionalSpouts.
      */
     @isString
@@ -1929,6 +1936,13 @@ public class Config extends HashMap<String, Object> {
     public static final String MULTITENANT_SCHEDULER_USER_POOLS = "multitenant.scheduler.user.pools";
 
     /**
+     * A map of users to another map of the resource guarantees of the user. Used by Resource Aware Scheduler to ensure
+     * per user resource guarantees.
+     */
+    @isMapEntryCustom(keyValidatorClasses = {StringValidator.class}, valueValidatorClasses = {UserResourcePoolEntryValidator.class})
+    public static final String RESOURCE_AWARE_SCHEDULER_USER_POOLS = "resource.aware.scheduler.user.pools";
+
+    /**
      * The number of machines that should be used by this topology to isolate it from all others. Set storm.scheduler
      * to backtype.storm.scheduler.multitenant.MultitenantScheduler
      */
@@ -2179,4 +2193,12 @@ public class Config extends HashMap<String, Object> {
             this.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, size);
         }
     }
+
+    /**
+     * set the priority for a topology
+     * @param priority
+     */
+    public void setTopologyPriority(int priority) {
+        this.put(Config.TOPOLOGY_PRIORITY, priority);
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/97b2c9a7/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
index 992a522..ac515b7 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
@@ -31,6 +31,7 @@ import backtype.storm.generated.SpoutSpec;
 import backtype.storm.generated.StormTopology;
 import backtype.storm.scheduler.resource.Component;
 import backtype.storm.scheduler.resource.ResourceUtils;
+import backtype.storm.utils.Time;
 import backtype.storm.utils.Utils;
 
 import org.slf4j.Logger;


[20/23] storm git commit: revisions based on comments

Posted by da...@apache.org.
revisions based on comments


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

Branch: refs/heads/master
Commit: dc6d0f797bd0da8d2da361378ae836cdcf61528e
Parents: 0f80d06
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Fri Dec 18 13:11:04 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Fri Dec 18 13:11:04 2015 -0600

----------------------------------------------------------------------
 .../resource/ResourceAwareScheduler.java        | 30 ++++++++++----------
 .../backtype/storm/scheduler/resource/User.java |  3 +-
 .../eviction/DefaultEvictionStrategy.java       | 19 ++++++++++++-
 3 files changed, 35 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/dc6d0f79/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
index 116f1b5..f97e6fe 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
@@ -83,13 +83,13 @@ public class ResourceAwareScheduler implements IScheduler {
     public void schedule(Topologies topologies, Cluster cluster) {
         LOG.debug("\n\n\nRerunning ResourceAwareScheduler...");
         //initialize data structures
-        this.initialize(topologies, cluster);
+        initialize(topologies, cluster);
         //logs everything that is currently scheduled and the location at which they are scheduled
         LOG.info("Cluster scheduling:\n{}", ResourceUtils.printScheduling(cluster, topologies));
         //logs the resources available/used for every node
         LOG.info("Nodes:\n{}", this.nodes);
         //logs the detailed info about each user
-        for (User user : this.getUserMap().values()) {
+        for (User user : getUserMap().values()) {
             LOG.info(user.getDetailedInfo());
         }
 
@@ -128,14 +128,14 @@ public class ResourceAwareScheduler implements IScheduler {
         if (cluster.getUnassignedExecutors(td).size() > 0) {
             LOG.debug("/********Scheduling topology {} from User {}************/", td.getName(), topologySubmitter);
 
-            SchedulingState schedulingState = this.checkpointSchedulingState();
+            SchedulingState schedulingState = checkpointSchedulingState();
             IStrategy rasStrategy = null;
             try {
                 rasStrategy = (IStrategy) Utils.newInstance((String) td.getConf().get(Config.TOPOLOGY_SCHEDULER_STRATEGY));
             } catch (RuntimeException e) {
                 LOG.error("failed to create instance of IStrategy: {} with error: {}! Topology {} will not be scheduled.",
                         td.getName(), td.getConf().get(Config.TOPOLOGY_SCHEDULER_STRATEGY), e.getMessage());
-                topologySubmitter = this.cleanup(schedulingState, td);
+                topologySubmitter = cleanup(schedulingState, td);
                 topologySubmitter.moveTopoFromPendingToInvalid(td);
                 this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling - failed to create instance of topology strategy "
                         + td.getConf().get(Config.TOPOLOGY_SCHEDULER_STRATEGY) + ". Please check logs for details");
@@ -151,7 +151,7 @@ public class ResourceAwareScheduler implements IScheduler {
                 } catch (Exception ex) {
                     LOG.error(String.format("Exception thrown when running strategy %s to schedule topology %s. Topology will not be scheduled!"
                             , rasStrategy.getClass().getName(), td.getName()), ex);
-                    topologySubmitter = this.cleanup(schedulingState, td);
+                    topologySubmitter = cleanup(schedulingState, td);
                     topologySubmitter.moveTopoFromPendingToInvalid(td);
                     this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling - Exception thrown when running strategy {}"
                             + rasStrategy.getClass().getName() + ". Please check logs for details");
@@ -170,7 +170,7 @@ public class ResourceAwareScheduler implements IScheduler {
                             }
                         } catch (IllegalStateException ex) {
                             LOG.error("Unsuccessful in scheduling - IllegalStateException thrown when attempting to assign executors to nodes.", ex);
-                            topologySubmitter = this.cleanup(schedulingState, td);
+                            topologySubmitter = cleanup(schedulingState, td);
                             topologySubmitter.moveTopoFromPendingToAttempted(td);
                             this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling - IllegalStateException thrown when attempting to assign executors to nodes. Please check log for details.");
                         }
@@ -195,31 +195,31 @@ public class ResourceAwareScheduler implements IScheduler {
                             } catch (Exception ex) {
                                 LOG.error(String.format("Exception thrown when running eviction strategy %s to schedule topology %s. No evictions will be done! Error: %s"
                                         , evictionStrategy.getClass().getName(), td.getName(), ex.getClass().getName()), ex);
-                                topologySubmitter = this.cleanup(schedulingState, td);
+                                topologySubmitter = cleanup(schedulingState, td);
                                 topologySubmitter.moveTopoFromPendingToAttempted(td);
                                 break;
                             }
                             if (!madeSpace) {
                                 LOG.debug("Could not make space for topo {} will move to attempted", td);
-                                topologySubmitter = this.cleanup(schedulingState, td);
+                                topologySubmitter = cleanup(schedulingState, td);
                                 topologySubmitter.moveTopoFromPendingToAttempted(td);
                                 this.cluster.setStatus(td.getId(), "Not enough resources to schedule - " + result.getErrorMessage());
                                 break;
                             }
                             continue;
                         } else if (result.getStatus() == SchedulingStatus.FAIL_INVALID_TOPOLOGY) {
-                            topologySubmitter = this.cleanup(schedulingState, td);
+                            topologySubmitter = cleanup(schedulingState, td);
                             topologySubmitter.moveTopoFromPendingToInvalid(td, this.cluster);
                             break;
                         } else {
-                            topologySubmitter = this.cleanup(schedulingState, td);
+                            topologySubmitter = cleanup(schedulingState, td);
                             topologySubmitter.moveTopoFromPendingToAttempted(td, this.cluster);
                             break;
                         }
                     }
                 } else {
                     LOG.warn("Scheduling results returned from topology {} is not vaild! Topology with be ignored.", td.getName());
-                    topologySubmitter = this.cleanup(schedulingState, td);
+                    topologySubmitter = cleanup(schedulingState, td);
                     topologySubmitter.moveTopoFromPendingToInvalid(td, this.cluster);
                     break;
                 }
@@ -234,7 +234,7 @@ public class ResourceAwareScheduler implements IScheduler {
     }
 
     private User cleanup(SchedulingState schedulingState, TopologyDetails td) {
-        this.restoreCheckpointSchedulingState(schedulingState);
+        restoreCheckpointSchedulingState(schedulingState);
         //since state is restored need the update User topologySubmitter to the new User object in userMap
         return this.userMap.get(td.getTopologySubmitter());
     }
@@ -310,7 +310,7 @@ public class ResourceAwareScheduler implements IScheduler {
      */
     private void initUsers(Topologies topologies, Cluster cluster) {
         this.userMap = new HashMap<String, User>();
-        Map<String, Map<String, Double>> userResourcePools = this.getUserResourcePools();
+        Map<String, Map<String, Double>> userResourcePools = getUserResourcePools();
         LOG.debug("userResourcePools: {}", userResourcePools);
 
         for (TopologyDetails td : topologies.getTopologies()) {
@@ -380,7 +380,7 @@ public class ResourceAwareScheduler implements IScheduler {
 
     private SchedulingState checkpointSchedulingState() {
         LOG.debug("/*********Checkpoint scheduling state************/");
-        for (User user : this.getUserMap().values()) {
+        for (User user : getUserMap().values()) {
             LOG.debug(user.getDetailedInfo());
         }
         LOG.debug(ResourceUtils.printScheduling(this.cluster, this.topologies));
@@ -404,7 +404,7 @@ public class ResourceAwareScheduler implements IScheduler {
         this.userMap = schedulingState.userMap;
         this.nodes = schedulingState.nodes;
 
-        for (User user : this.getUserMap().values()) {
+        for (User user : getUserMap().values()) {
             LOG.debug(user.getDetailedInfo());
         }
         LOG.debug(ResourceUtils.printScheduling(cluster, topologies));

http://git-wip-us.apache.org/repos/asf/storm/blob/dc6d0f79/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
index f1d53c6..f3434ac 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
@@ -38,9 +38,10 @@ public class User {
     //Topologies yet to be scheduled sorted by priority for each user
     private TreeSet<TopologyDetails> runningQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
 
-    //Topologies that was attempted to be scheduled but wasn't successull
+    //Topologies that was attempted to be scheduled but wasn't successful
     private TreeSet<TopologyDetails> attemptedQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
 
+    //Topologies that was deemed to be invalid
     private TreeSet<TopologyDetails> invalidQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
 
     private Map<String, Double> resourcePool = new HashMap<String, Double>();

http://git-wip-us.apache.org/repos/asf/storm/blob/dc6d0f79/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
index d54ec43..1f446bd 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
@@ -62,6 +62,12 @@ public class DefaultEvictionStrategy implements IEvictionStrategy {
         if ((1.0 - submitter.getCPUResourcePoolUtilization()) >= cpuNeeded && (1.0 - submitter.getMemoryResourcePoolUtilization()) >= memoryNeeded) {
             if (evictUser != null) {
                 TopologyDetails topologyEvict = evictUser.getRunningTopologyWithLowestPriority();
+                LOG.debug("Running Topology {} from user {} is still within user's resource guarantee thus, POTENTIALLY evicting Topology {} from user {} since:" +
+                                "\n(1.0 - submitter.getCPUResourcePoolUtilization()) = {} >= cpuNeeded = {}" +
+                                "\nand" +
+                                "\n(1.0 - submitter.getMemoryResourcePoolUtilization()) = {} >= memoryNeeded = {}"
+                        ,td, submitter, topologyEvict, evictUser, (1.0 - submitter.getCPUResourcePoolUtilization())
+                        , cpuNeeded, (1.0 - submitter.getMemoryResourcePoolUtilization()), memoryNeeded);
                 evictTopology(topologyEvict);
                 return true;
             }
@@ -69,16 +75,27 @@ public class DefaultEvictionStrategy implements IEvictionStrategy {
             if (evictUser != null) {
                 if ((evictUser.getResourcePoolAverageUtilization() - 1.0) > (((cpuNeeded + memoryNeeded) / 2) + (submitter.getResourcePoolAverageUtilization() - 1.0))) {
                     TopologyDetails topologyEvict = evictUser.getRunningTopologyWithLowestPriority();
+                    LOG.debug("POTENTIALLY Evicting Topology {} from user {} since:" +
+                                    "\n((evictUser.getResourcePoolAverageUtilization() - 1.0) = {}" +
+                                    "\n(cpuNeeded + memoryNeeded) / 2) = {} and (submitter.getResourcePoolAverageUtilization() - 1.0)) = {} Thus," +
+                                    "\n(evictUser.getResourcePoolAverageUtilization() - 1.0) = {} > (((cpuNeeded + memoryNeeded) / 2) + (submitter.getResourcePoolAverageUtilization() - 1.0)) = {}"
+                            ,topologyEvict, evictUser, (evictUser.getResourcePoolAverageUtilization() - 1.0), ((cpuNeeded + memoryNeeded) / 2)
+                            , (submitter.getResourcePoolAverageUtilization() - 1.0), (evictUser.getResourcePoolAverageUtilization() - 1.0)
+                            , (((cpuNeeded + memoryNeeded) / 2) + (submitter.getResourcePoolAverageUtilization() - 1.0)));
                     evictTopology(topologyEvict);
                     return true;
                 }
             }
         }
         //See if there is a lower priority topology that can be evicted from the current user
+        //topologies should already be sorted in order of increasing priority.
+        //Thus, topology at the front of the queue has the lowest priority
         for (TopologyDetails topo : submitter.getTopologiesRunning()) {
             //check to if there is a topology with a lower priority we can evict
             if (topo.getTopologyPriority() > td.getTopologyPriority()) {
-                evictTopology(topo);
+                LOG.debug("POTENTIALLY Evicting Topology {} from user {} (itself) since topology {} has a lower priority than topology {}"
+                        , topo, submitter, topo, td);
+                        evictTopology(topo);
                 return true;
             }
         }


[11/23] storm git commit: fixing simple issues based on comments

Posted by da...@apache.org.
fixing simple issues based on comments


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/88ad3c31
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/88ad3c31
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/88ad3c31

Branch: refs/heads/master
Commit: 88ad3c316a5a0df7c083470fb66847e5fb29d663
Parents: 3e83220
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Fri Dec 4 14:02:56 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Fri Dec 4 14:02:56 2015 -0600

----------------------------------------------------------------------
 .../storm/starter/ResourceAwareExampleTopology.java  |  4 +---
 storm-core/src/jvm/backtype/storm/Config.java        |  1 +
 .../src/jvm/backtype/storm/scheduler/Cluster.java    |  4 +---
 .../src/jvm/backtype/storm/scheduler/Topologies.java |  7 ++++---
 .../backtype/storm/scheduler/TopologyDetails.java    | 10 +++++++++-
 .../backtype/storm/scheduler/resource/RAS_Node.java  |  6 +++---
 .../scheduler/resource/ResourceAwareScheduler.java   | 12 ++++++------
 .../storm/scheduler/resource/SchedulingResult.java   | 15 ++++++++++-----
 .../jvm/backtype/storm/scheduler/resource/User.java  | 14 ++++++++------
 .../strategies/eviction/DefaultEvictionStrategy.java |  5 ++---
 10 files changed, 45 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/88ad3c31/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java b/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
index a9ac659..7104281 100644
--- a/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
+++ b/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
@@ -54,8 +54,6 @@ public class ResourceAwareExampleTopology {
     public void declareOutputFields(OutputFieldsDeclarer declarer) {
       declarer.declare(new Fields("word"));
     }
-
-
   }
 
   public static void main(String[] args) throws Exception {
@@ -87,7 +85,7 @@ public class ResourceAwareExampleTopology {
     // Set topology priority 0-30 with 0 being the highest priority and 30 being the lowest priority.
     conf.setTopologyPriority(30);
 
-    //Set strategy to schedule topology. If not specified, default to backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy
+    // Set strategy to schedule topology. If not specified, default to backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy
     conf.setTopologyStrategy(backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class);
 
     if (args != null && args.length > 0) {

http://git-wip-us.apache.org/repos/asf/storm/blob/88ad3c31/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index f3c8c4a..dacf4f8 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -2229,6 +2229,7 @@ public class Config extends HashMap<String, Object> {
 
     /**
      * Takes as input the strategy class name. Strategy must implement the IStrategy interface
+     * @param clazz class of the strategy to use
      */
     public void setTopologyStrategy(Class<? extends IStrategy> clazz) {
         if (clazz != null) {

http://git-wip-us.apache.org/repos/asf/storm/blob/88ad3c31/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
index c35dbbd..92b2219 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
@@ -104,9 +104,7 @@ public class Cluster {
         Map newConf = new HashMap<String, Object>();
         newConf.putAll(this.conf);
         Cluster copy = new Cluster(this.inimbus, this.supervisors, newAssignments, newConf);
-        for (Map.Entry<String, String> entry : this.status.entrySet()) {
-            copy.setStatus(entry.getKey(), entry.getValue());
-        }
+        copy.status = new HashMap<>(this.status);
         return copy;
     }
     

http://git-wip-us.apache.org/repos/asf/storm/blob/88ad3c31/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
index 3a6361f..b6fbd07 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
@@ -74,10 +74,11 @@ public class Topologies {
 
     @Override
     public String toString() {
-        String ret = "Topologies:\n";
+        StringBuilder ret = new StringBuilder();
+        ret.append("Topologies:\n");
         for (TopologyDetails td : this.getTopologies()) {
-            ret += td.toString() + "\n";
+            ret.append(td.toString()).append("\n");
         }
-        return ret;
+        return ret.toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/88ad3c31/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
index 871ae9b..166493f 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
@@ -469,7 +469,7 @@ public class TopologyDetails {
         String user = (String) this.topologyConf.get(Config.TOPOLOGY_SUBMITTER_USER);
         if (user == null || user.equals("")) {
             LOG.debug("Topology {} submitted by anonymous user", this.getName());
-            user = "anonymous";
+            user = System.getProperty("user.name");
         }
         return user;
     }
@@ -506,4 +506,12 @@ public class TopologyDetails {
     public int hashCode() {
         return this.topologyId.hashCode();
     }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof TopologyDetails)) {
+            return false;
+        }
+        return (this.topologyId.equals(((TopologyDetails) o).getId()));
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/88ad3c31/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
index 54775bf..8d0df62 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
@@ -144,7 +144,7 @@ public class RAS_Node {
         }
     }
 
-     void addOrphanedSlot(WorkerSlot ws) {
+    void addOrphanedSlot(WorkerSlot ws) {
         if (_isAlive) {
             throw new IllegalArgumentException("Orphaned Slots " +
                     "only are allowed on dead nodes.");
@@ -241,7 +241,7 @@ public class RAS_Node {
         _topIdToUsedSlots.remove(topId);
     }
 
-    public void freeMemory(double amount) {
+    private void freeMemory(double amount) {
         _availMemory += amount;
         LOG.debug("freeing {} memory on node {}...avail mem: {}", amount, this.getHostname(), _availMemory);
         if (_availMemory > this.getTotalMemoryResources()) {
@@ -249,7 +249,7 @@ public class RAS_Node {
         }
     }
 
-    public void freeCPU(double amount) {
+    private void freeCPU(double amount) {
         _availCPU += amount;
         LOG.debug("freeing {} CPU on node...avail CPU: {}", amount, this.getHostname(), _availCPU);
         if (_availCPU > this.getAvailableCpuResources()) {

http://git-wip-us.apache.org/repos/asf/storm/blob/88ad3c31/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
index 53672f6..c2e2fcd 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
@@ -131,9 +131,9 @@ public class ResourceAwareScheduler implements IScheduler {
             LOG.debug("/********Scheduling topology {} from User {}************/", td.getName(), topologySubmitter);
 
             SchedulingState schedulingState = this.checkpointSchedulingState();
-            IStrategy RAStrategy = null;
+            IStrategy rasStrategy = null;
             try {
-                RAStrategy = (IStrategy) Utils.newInstance((String) td.getConf().get(Config.TOPOLOGY_SCHEDULER_STRATEGY));
+                rasStrategy = (IStrategy) Utils.newInstance((String) td.getConf().get(Config.TOPOLOGY_SCHEDULER_STRATEGY));
             } catch (RuntimeException e) {
                 LOG.error("failed to create instance of IStrategy: {} with error: {}! Topology {} will not be scheduled.",
                         td.getName(), td.getConf().get(Config.TOPOLOGY_SCHEDULER_STRATEGY), e.getMessage());
@@ -150,17 +150,17 @@ public class ResourceAwareScheduler implements IScheduler {
                 SchedulingResult result = null;
                 try {
                     //Need to re prepare scheduling strategy with cluster and topologies in case scheduling state was restored
-                    RAStrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
-                    result = RAStrategy.schedule(td);
+                    rasStrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
+                    result = rasStrategy.schedule(td);
                 } catch (Exception e) {
                     LOG.error("Exception thrown when running strategy {} to schedule topology {}. Topology will not be scheduled! Error: {} StackTrack: {}"
-                            , RAStrategy.getClass().getName(), td.getName(), e.getMessage(), Arrays.toString(e.getStackTrace()));
+                            , rasStrategy.getClass().getName(), td.getName(), e.getMessage(), Arrays.toString(e.getStackTrace()));
                     this.restoreCheckpointSchedulingState(schedulingState);
                     //since state is restored need the update User topologySubmitter to the new User object in userMap
                     topologySubmitter = this.userMap.get(td.getTopologySubmitter());
                     topologySubmitter.moveTopoFromPendingToInvalid(td);
                     this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling - Exception thrown when running strategy {}"
-                            + RAStrategy.getClass().getName() + ". Please check logs for details");
+                            + rasStrategy.getClass().getName() + ". Please check logs for details");
                 }
                 LOG.debug("scheduling result: {}", result);
                 if (result != null && result.isValid()) {

http://git-wip-us.apache.org/repos/asf/storm/blob/88ad3c31/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingResult.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingResult.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingResult.java
index 9e7b1ff..13ed8ad 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingResult.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingResult.java
@@ -26,21 +26,26 @@ import org.slf4j.LoggerFactory;
 import java.util.Collection;
 import java.util.Map;
 
+/**
+ * This class serves as a mechanism to return results and messages from a scheduling strategy to the Resource Aware Scheduler
+ */
 public class SchedulingResult {
 
     //contains the result for the attempted scheduling
     private Map<WorkerSlot, Collection<ExecutorDetails>> schedulingResultMap = null;
 
+    //status of scheduling the topology e.g. success or fail?
     private SchedulingStatus status = null;
 
+    //arbitrary message to be returned when scheduling is done
     private String message = null;
 
+    //error message returned is something went wrong
     private String errorMessage = null;
 
     private static final Logger LOG = LoggerFactory.getLogger(SchedulingResult.class);
 
-
-    public SchedulingResult(SchedulingStatus status, Map<WorkerSlot, Collection<ExecutorDetails>> schedulingResultMap, String message, String errorMessage) {
+    private SchedulingResult(SchedulingStatus status, Map<WorkerSlot, Collection<ExecutorDetails>> schedulingResultMap, String message, String errorMessage) {
         this.status = status;
         this.schedulingResultMap = schedulingResultMap;
         this.message = message;
@@ -100,11 +105,11 @@ public class SchedulingResult {
 
     @Override
     public String toString() {
-        String ret = "";
+        String ret = null;
         if(this.isSuccess()) {
-            ret += "Status: " + this.getStatus() + " message: " + this.getMessage() + " scheduling: " + this.getSchedulingResultMap().toString();
+            ret = "Status: " + this.getStatus() + " message: " + this.getMessage() + " scheduling: " + this.getSchedulingResultMap();
         } else {
-            ret += "Status: " + this.getStatus() + " error message: " + this.getErrorMessage();
+            ret = "Status: " + this.getStatus() + " error message: " + this.getErrorMessage();
         }
         return ret;
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/88ad3c31/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
index 8542120..7f49446 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
@@ -159,7 +159,6 @@ public class User {
         this.moveTopoFromPendingToRunning(topo, null);
     }
 
-
     public void moveTopoFromPendingToAttempted(TopologyDetails topo, Cluster cluster) {
         moveTopology(topo, this.pendingQueue, "pending", this.attemptedQueue, "attempted");
         if (cluster != null) {
@@ -171,7 +170,6 @@ public class User {
         this.moveTopoFromPendingToAttempted(topo, null);
     }
 
-
     public void moveTopoFromPendingToInvalid(TopologyDetails topo, Cluster cluster) {
         moveTopology(topo, this.pendingQueue, "pending", this.invalidQueue, "invalid");
         if (cluster != null) {
@@ -183,7 +181,6 @@ public class User {
         this.moveTopoFromPendingToInvalid(topo, null);
     }
 
-
     public void moveTopoFromRunningToPending(TopologyDetails topo, Cluster cluster) {
         moveTopology(topo, this.runningQueue, "running", this.pendingQueue, "pending");
         if (cluster != null) {
@@ -195,7 +192,6 @@ public class User {
         this.moveTopoFromRunningToPending(topo, null);
     }
 
-
     private void moveTopology(TopologyDetails topo, Set<TopologyDetails> src, String srcName, Set<TopologyDetails> dest, String destName) {
         LOG.debug("For User {} Moving topo {} from {} to {}", this.userId, topo.getName(), srcName, destName);
         if (topo == null) {
@@ -213,7 +209,6 @@ public class User {
         dest.add(topo);
     }
 
-
     public double getResourcePoolAverageUtilization() {
         Double cpuResourcePoolUtilization = this.getCPUResourcePoolUtilization();
         Double memoryResourcePoolUtilization = this.getMemoryResourcePoolUtilization();
@@ -243,7 +238,6 @@ public class User {
         return this.getMemoryResourceUsedByUser() / memoryGuarantee;
     }
 
-
     public double getCPUResourceUsedByUser() {
         double sum = 0.0;
         for (TopologyDetails topo : this.runningQueue) {
@@ -294,6 +288,14 @@ public class User {
     }
 
     @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof User)) {
+            return false;
+        }
+        return this.getId().equals(((User) o).getId());
+    }
+
+    @Override
     public String toString() {
         return this.userId;
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/88ad3c31/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
index f0401ce..81c2abc 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
@@ -23,7 +23,6 @@ import backtype.storm.scheduler.Topologies;
 import backtype.storm.scheduler.TopologyDetails;
 import backtype.storm.scheduler.WorkerSlot;
 import backtype.storm.scheduler.resource.RAS_Nodes;
-import backtype.storm.scheduler.resource.ResourceUtils;
 import backtype.storm.scheduler.resource.User;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -58,7 +57,7 @@ public class DefaultEvictionStrategy implements IEvictionStrategy {
         double cpuNeeded = td.getTotalRequestedCpu() / submitter.getCPUResourceGuaranteed();
         double memoryNeeded = (td.getTotalRequestedMemOffHeap() + td.getTotalRequestedMemOnHeap()) / submitter.getMemoryResourceGuaranteed();
 
-        User evictUser = this.findUserWithMostResourcesAboveGuarantee();
+        User evictUser = this.findUserWithHighestAverageResourceUtilAboveGuarantee();
         //user has enough resource under his or her resource guarantee to schedule topology
         if ((1.0 - submitter.getCPUResourcePoolUtilization()) >= cpuNeeded && (1.0 - submitter.getMemoryResourcePoolUtilization()) >= memoryNeeded) {
             if (evictUser != null) {
@@ -96,7 +95,7 @@ public class DefaultEvictionStrategy implements IEvictionStrategy {
         submitter.moveTopoFromRunningToPending(topologyEvict, this.cluster);
     }
 
-    private User findUserWithMostResourcesAboveGuarantee() {
+    private User findUserWithHighestAverageResourceUtilAboveGuarantee() {
         double most = 0.0;
         User mostOverUser = null;
         for (User user : this.userMap.values()) {


[22/23] storm git commit: adds STORM-898

Posted by da...@apache.org.
adds STORM-898


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

Branch: refs/heads/master
Commit: eaf5b37aaa176d61ad3104d6abd76fde0d8890e8
Parents: d790eb5
Author: Derek Dagit <de...@yahoo-inc.com>
Authored: Mon Dec 21 08:28:23 2015 -0600
Committer: Derek Dagit <de...@yahoo-inc.com>
Committed: Mon Dec 21 08:28:23 2015 -0600

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/eaf5b37a/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 5213e44..a084de7 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.11.0
+ * STORM-898: Add priorities and per user resource guarantees to Resource Aware Scheduler
  * STORM-1187: Support windowing based on tuple ts.
  * STORM-1400: Netty Context removeClient() called after term() causes NullPointerException.
  * STORM-1383: Supervisors should not crash if nimbus is unavailable


[06/23] storm git commit: adding checkpointing

Posted by da...@apache.org.
adding checkpointing


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

Branch: refs/heads/master
Commit: 9d3c864cdc708d62a088a1b9b23904bd6ea1b276
Parents: 3f55fee
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Wed Nov 18 11:09:45 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Fri Dec 4 13:07:00 2015 -0600

----------------------------------------------------------------------
 .../jvm/backtype/storm/scheduler/Cluster.java   |   8 +
 .../backtype/storm/scheduler/Topologies.java    |   4 +
 .../storm/scheduler/resource/RAS_Node.java      |  77 +++-
 .../storm/scheduler/resource/RAS_Nodes.java     |  29 +-
 .../resource/ResourceAwareScheduler.java        | 236 ++++++-----
 .../storm/scheduler/resource/ResourceUtils.java |   9 +
 .../backtype/storm/scheduler/resource/User.java |  52 ++-
 .../storm/scheduler/resource/Experiment.java    | 257 +++++++-----
 .../resource/TestResourceAwareScheduler.java    | 400 ++++++++++++++++++-
 .../TestUtilsForResourceAwareScheduler.java     |  28 ++
 10 files changed, 862 insertions(+), 238 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/9d3c864c/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
index 7f16b86..2676af1 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
@@ -90,6 +90,14 @@ public class Cluster {
         }
         this.conf = storm_conf;
     }
+
+    public Cluster getCopy() {
+        Cluster copy = new Cluster(this.inimbus, this.supervisors, this.assignments, this.conf);
+        for(Map.Entry<String, String> entry : this.status.entrySet()) {
+            copy.setStatus(entry.getKey(), entry.getValue());
+        }
+        return copy;
+    }
     
     public void setBlacklistedHosts(Set<String> hosts) {
         blackListedHosts = hosts;

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3c864c/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
index 0b4c0ca..59a53c8 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
@@ -68,6 +68,10 @@ public class Topologies {
         return _allComponents;
     }
 
+    public Topologies getCopy() {
+        return new Topologies(this.topologies);
+    }
+
     @Override
     public String toString() {
         String ret = "Topologies:\n";

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3c864c/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
index b0bcc8a..21367f0 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
@@ -55,9 +55,10 @@ public class RAS_Node {
     private Double _availCPU;
     private List<WorkerSlot> _slots;
     private Cluster _cluster;
+    private Topologies _topologies;
 
     public RAS_Node(String nodeId, Set<Integer> allPorts, boolean isAlive,
-                    SupervisorDetails sup, Cluster cluster) {
+                    SupervisorDetails sup, Cluster cluster, Topologies topologies) {
         _slots = new ArrayList<WorkerSlot>();
         _nodeId = nodeId;
         _isAlive = isAlive;
@@ -72,6 +73,7 @@ public class RAS_Node {
             _slots.addAll(_freeSlots);
         }
         this._cluster = cluster;
+        this._topologies = topologies;
     }
 
     public String getId() {
@@ -185,6 +187,8 @@ public class RAS_Node {
         }
         for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
             _cluster.freeSlots(entry.getValue());
+            _availCPU = this.getTotalCpuResources();
+            _availMemory = this.getAvailableMemoryResources();
             if (_isAlive) {
                 _freeSlots.addAll(entry.getValue());
             }
@@ -197,14 +201,19 @@ public class RAS_Node {
      * @param ws the slot to free
      */
     public void free(WorkerSlot ws) {
+        LOG.info("freeing ws {} on node {}", ws, _hostname);
         if (_freeSlots.contains(ws)) return;
         for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
             Set<WorkerSlot> slots = entry.getValue();
+            double memUsed = this.getMemoryUsedByWorker(ws);
+            double cpuUsed = this.getCpuUsedByWorker(ws);
             if (slots.remove(ws)) {
                 _cluster.freeSlot(ws);
                 if (_isAlive) {
                     _freeSlots.add(ws);
                 }
+                this.freeMemory(memUsed);
+                this.freeCPU(cpuUsed);
                 return;
             }
         }
@@ -223,6 +232,8 @@ public class RAS_Node {
         }
         for (WorkerSlot ws : slots) {
             _cluster.freeSlot(ws);
+            this.freeMemory(this.getMemoryUsedByWorker(ws));
+            this.freeCPU(this.getCpuUsedByWorker(ws));
             if (_isAlive) {
                 _freeSlots.add(ws);
             }
@@ -230,6 +241,66 @@ public class RAS_Node {
         _topIdToUsedSlots.remove(topId);
     }
 
+    public void freeMemory(double amount) {
+        _availMemory += amount;
+        LOG.info("freeing {} memory...avail mem: {}", amount, _availMemory);
+        if(_availMemory > this.getTotalMemoryResources()) {
+            LOG.warn("Freeing more memory than there exists!");
+        }
+    }
+
+    public void freeCPU(double amount) {
+        _availCPU += amount;
+        LOG.info("freeing {} CPU...avail CPU: {}", amount, _availCPU);
+        if(_availCPU > this.getAvailableCpuResources()) {
+            LOG.warn("Freeing more memory than there exists!");
+        }
+    }
+
+    public double getMemoryUsedByWorker(WorkerSlot ws) {
+        TopologyDetails topo = this.findTopologyUsingWorker(ws);
+        LOG.info("Topology {} using worker {}", topo, ws);
+        if(topo == null) {
+            return 0.0;
+        }
+        Collection<ExecutorDetails> execs = this.getExecutors(ws, this._cluster);
+        LOG.info("Worker {} has execs: {}", ws, execs);
+        double totalMemoryUsed = 0.0;
+        for(ExecutorDetails exec : execs) {
+            totalMemoryUsed += topo.getTotalMemReqTask(exec);
+        }
+        return totalMemoryUsed;
+    }
+
+    public double getCpuUsedByWorker(WorkerSlot ws) {
+        TopologyDetails topo = this.findTopologyUsingWorker(ws);
+        LOG.info("Topology {} using worker {}", topo, ws);
+        if(topo == null) {
+            return 0.0;
+        }
+        Collection<ExecutorDetails> execs = this.getExecutors(ws, this._cluster);
+        LOG.info("Worker {} has execs: {}", ws, execs);
+        double totalCpuUsed = 0.0;
+        for(ExecutorDetails exec : execs) {
+            totalCpuUsed += topo.getTotalCpuReqTask(exec);
+        }
+        return totalCpuUsed;
+    }
+
+    public TopologyDetails findTopologyUsingWorker(WorkerSlot ws) {
+        for(Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
+            LOG.info("topoId: {} workers: {}", entry.getKey(), entry.getValue());
+            String topoId = entry.getKey();
+            Set<WorkerSlot> workers = entry.getValue();
+            for (WorkerSlot worker : workers) {
+                if(worker.getNodeId().equals(ws.getNodeId()) && worker.getPort() == ws.getPort()) {
+                    return _topologies.getById(topoId);
+                }
+            }
+        }
+        return null;
+    }
+
     /**
      * Allocate Mem and CPU resources to the assigned slot for the topology's executors.
      * @param td the TopologyDetails that the slot is assigned to.
@@ -457,4 +528,8 @@ public class RAS_Node {
         this.consumeCPU(taskCpuReq);
         this.consumeMemory(taskMemReq);
     }
+
+    public Map<String, Set<WorkerSlot>> getTopoIdTousedSlots() {
+        return _topIdToUsedSlots;
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3c864c/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
index 9df1475..42fc236 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
@@ -56,7 +56,7 @@ public class RAS_Nodes {
             LOG.debug("Found a {} Node {} {}",
                     isAlive ? "living" : "dead", id, sup.getAllPorts());
             LOG.debug("resources_mem: {}, resources_CPU: {}", sup.getTotalMemory(), sup.getTotalCPU());
-            nodeIdToNode.put(sup.getId(), new RAS_Node(id, sup.getAllPorts(), isAlive, sup, cluster));
+            nodeIdToNode.put(sup.getId(), new RAS_Node(id, sup.getAllPorts(), isAlive, sup, cluster, topologies));
         }
         for (Map.Entry<String, SchedulerAssignment> entry : cluster.getAssignments().entrySet()) {
             String topId = entry.getValue().getTopologyId();
@@ -66,7 +66,7 @@ public class RAS_Nodes {
                 if (node == null) {
                     LOG.info("Found an assigned slot on a dead supervisor {} with executors {}",
                             workerSlot, RAS_Node.getExecutors(workerSlot, cluster));
-                    node = new RAS_Node(id, null, false, null, cluster);
+                    node = new RAS_Node(id, null, false, null, cluster, topologies);
                     nodeIdToNode.put(id, node);
                 }
                 if (!node.isAlive()) {
@@ -86,8 +86,9 @@ public class RAS_Nodes {
     /**
      * updates the available resources for every node in a cluster
      * by recalculating memory requirements.
-     * @param cluster the cluster used in this calculation
-     * @param topologies container of all topologies
+     *
+     * @param cluster      the cluster used in this calculation
+     * @param topologies   container of all topologies
      * @param nodeIdToNode a map between node id and node
      */
     private static void updateAvailableResources(Cluster cluster,
@@ -138,12 +139,26 @@ public class RAS_Nodes {
     }
 
     public void freeSlots(Collection<WorkerSlot> workerSlots) {
-        for(RAS_Node node : nodeMap.values()) {
-            for(WorkerSlot ws : node.getUsedSlots()) {
-                if(workerSlots.contains(ws)) {
+        for (RAS_Node node : nodeMap.values()) {
+            for (WorkerSlot ws : node.getUsedSlots()) {
+                if (workerSlots.contains(ws)) {
+                    LOG.info("freeing ws {} on node {}", ws, node);
                     node.free(ws);
                 }
             }
         }
     }
+
+    public Collection<RAS_Node> getNodes() {
+        return this.nodeMap.values();
+    }
+
+    @Override
+    public String toString() {
+        String ret = "";
+        for (RAS_Node node : this.nodeMap.values()) {
+            ret += node.toString() + "\n";
+        }
+        return ret;
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3c864c/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
index 279d060..934858e 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
@@ -45,6 +45,27 @@ public class ResourceAwareScheduler implements IScheduler {
     private Topologies topologies;
     private RAS_Nodes nodes;
 
+    private class SchedulingState {
+        private Map<String, User> userMap = new HashMap<String, User>();
+        private Cluster cluster;
+        private Topologies topologies;
+        private RAS_Nodes nodes;
+        private Map conf = new Config();
+
+        public SchedulingState(Map<String, User> userMap, Cluster cluster, Topologies topologies, RAS_Nodes nodes, Map conf) {
+            for(Map.Entry<String, User> userMapEntry : userMap.entrySet()) {
+                String userId = userMapEntry.getKey();
+                User user = userMapEntry.getValue();
+                this.userMap.put(userId, user.getCopy());
+            }
+            this.cluster = cluster.getCopy();
+            this.topologies = topologies.getCopy();
+            this.nodes = new RAS_Nodes(this.cluster, this.topologies);
+            this.conf.putAll(conf);
+
+        }
+    }
+
 
     @SuppressWarnings("rawtypes")
     private Map conf;
@@ -67,27 +88,35 @@ public class ResourceAwareScheduler implements IScheduler {
         this.initialize(topologies, cluster);
 
         LOG.info("UserMap:\n{}", this.userMap);
-        for(User user : this.getUserMap().values()) {
+        for (User user : this.getUserMap().values()) {
             LOG.info(user.getDetailedInfo());
         }
 
-        for(TopologyDetails topo : topologies.getTopologies()) {
+        for (TopologyDetails topo : topologies.getTopologies()) {
             LOG.info("topo {} status: {}", topo, cluster.getStatusMap().get(topo.getId()));
         }
 
+        LOG.info("Nodes:\n{}", this.nodes);
+
         LOG.info("getNextUser: {}", this.getNextUser());
 
-        while(true) {
+        while (true) {
+            LOG.info("/*********** next scheduling iteration **************/");
+
             User nextUser = this.getNextUser();
-            if(nextUser == null){
+            if (nextUser == null) {
                 break;
             }
             TopologyDetails td = nextUser.getNextTopologyToSchedule();
             scheduleTopology(td);
         }
+        //since scheduling state might have been restored thus need to set the cluster and topologies.
+        cluster = this.cluster;
+        topologies = this.topologies;
     }
 
     private boolean makeSpaceForTopo(TopologyDetails td) {
+        LOG.info("attempting to make space for topo {} from user {}", td.getName(), td.getTopologySubmitter());
         User submitter = this.userMap.get(td.getTopologySubmitter());
         if (submitter.getCPUResourceGuaranteed() == null || submitter.getMemoryResourceGuaranteed() == null) {
             return false;
@@ -97,7 +126,7 @@ public class ResourceAwareScheduler implements IScheduler {
         double memoryNeeded = (td.getTotalRequestedMemOffHeap() + td.getTotalRequestedMemOnHeap()) / submitter.getMemoryResourceGuaranteed();
 
         //user has enough resource under his or her resource guarantee to schedule topology
-        if ((1.0 - submitter.getCPUResourcePoolUtilization()) > cpuNeeded && (1.0 - submitter.getMemoryResourcePoolUtilization()) > memoryNeeded) {
+        if ((1.0 - submitter.getCPUResourcePoolUtilization()) >= cpuNeeded && (1.0 - submitter.getMemoryResourcePoolUtilization()) >= memoryNeeded) {
             User evictUser = this.findUserWithMostResourcesAboveGuarantee();
             if (evictUser == null) {
                 LOG.info("Cannot make space for topology {} from user {}", td.getName(), submitter.getId());
@@ -109,6 +138,8 @@ public class ResourceAwareScheduler implements IScheduler {
             LOG.info("topology to evict: {}", topologyEvict);
             evictTopology(topologyEvict);
 
+            LOG.info("Resources After eviction:\n{}", this.nodes);
+
             return true;
         } else {
 
@@ -129,6 +160,7 @@ public class ResourceAwareScheduler implements IScheduler {
         User submitter = this.userMap.get(topologyEvict.getTopologySubmitter());
 
         LOG.info("Evicting Topology {} with workers: {}", topologyEvict.getName(), workersToEvict);
+        LOG.debug("From Nodes:\n{}", ResourceUtils.printScheduling(this.nodes));
         this.nodes.freeSlots(workersToEvict);
         submitter.moveTopoFromRunningToPending(topologyEvict, this.cluster);
         LOG.info("check if topology unassigned: {}", this.cluster.getUsedSlotsByTopologyId(topologyEvict.getId()));
@@ -137,9 +169,9 @@ public class ResourceAwareScheduler implements IScheduler {
     private User findUserWithMostResourcesAboveGuarantee() {
         double most = 0.0;
         User mostOverUser = null;
-        for(User user : this.userMap.values()) {
-            double over = user.getResourcePoolAverageUtilization() -1.0;
-            if((over > most) && (!user.getTopologiesRunning().isEmpty())) {
+        for (User user : this.userMap.values()) {
+            double over = user.getResourcePoolAverageUtilization() - 1.0;
+            if ((over > most) && (!user.getTopologiesRunning().isEmpty())) {
                 most = over;
                 mostOverUser = user;
             }
@@ -147,63 +179,74 @@ public class ResourceAwareScheduler implements IScheduler {
         return mostOverUser;
     }
 
-    public void resetAssignments(Map<String, SchedulerAssignment> assignmentCheckpoint) {
-        this.cluster.setAssignments(assignmentCheckpoint);
-    }
-
     public void scheduleTopology(TopologyDetails td) {
-        ResourceAwareStrategy RAStrategy = new ResourceAwareStrategy(this.cluster, this.topologies);
         User topologySubmitter = this.userMap.get(td.getTopologySubmitter());
         if (cluster.getUnassignedExecutors(td).size() > 0) {
             LOG.info("/********Scheduling topology {} from User {}************/", td.getName(), topologySubmitter);
             LOG.info("{}", this.userMap.get(td.getTopologySubmitter()).getDetailedInfo());
             LOG.info("{}", User.getResourcePoolAverageUtilizationForUsers(this.userMap.values()));
+            LOG.info("Nodes:\n{}", this.nodes);
+            LOG.debug("From cluster:\n{}", ResourceUtils.printScheduling(this.cluster, this.topologies));
+            LOG.debug("From Nodes:\n{}", ResourceUtils.printScheduling(this.nodes));
 
-            Map<String, SchedulerAssignment> assignmentCheckpoint = this.cluster.getAssignments();
-
+            SchedulingState schedulingState = this.checkpointSchedulingState();
             while (true) {
+                //Need to reinitialize ResourceAwareStrategy with cluster and topologies in case scheduling state was restored
+                ResourceAwareStrategy RAStrategy = new ResourceAwareStrategy(this.cluster, this.topologies);
                 SchedulingResult result = RAStrategy.schedule(td);
                 LOG.info("scheduling result: {}", result);
                 if (result.isValid()) {
                     if (result.isSuccess()) {
                         try {
-                            if(mkAssignment(td, result.getSchedulingResultMap())) {
+                            if (mkAssignment(td, result.getSchedulingResultMap())) {
                                 topologySubmitter.moveTopoFromPendingToRunning(td, this.cluster);
                             } else {
-                                resetAssignments(assignmentCheckpoint);
+                                //resetAssignments(assignmentCheckpoint);
+                                this.restoreCheckpointSchedulingState(schedulingState);
+                                topologySubmitter = this.userMap.get(td.getTopologySubmitter());
                                 topologySubmitter.moveTopoFromPendingToAttempted(td, this.cluster);
                             }
                         } catch (IllegalStateException ex) {
                             LOG.error(ex.toString());
-                            LOG.error("Unsuccessful in scheduling", td.getId());
+                            LOG.error("Unsuccessful in scheduling: IllegalStateException thrown!", td.getId());
                             this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
-                            resetAssignments(assignmentCheckpoint);
+                            this.restoreCheckpointSchedulingState(schedulingState);
+                            //since state is restored need the update User topologySubmitter to the new User object in userMap
+                            topologySubmitter = this.userMap.get(td.getTopologySubmitter());
                             topologySubmitter.moveTopoFromPendingToAttempted(td, this.cluster);
                         }
                         break;
                     } else {
                         if (result.getStatus() == SchedulingStatus.FAIL_NOT_ENOUGH_RESOURCES) {
-                            if(!this.makeSpaceForTopo(td)) {
-                                topologySubmitter.moveTopoFromPendingToAttempted(td);
+                            if (!this.makeSpaceForTopo(td)) {
                                 this.cluster.setStatus(td.getId(), result.getErrorMessage());
-                                resetAssignments(assignmentCheckpoint);
+                                this.restoreCheckpointSchedulingState(schedulingState);
+                                //since state is restored need the update User topologySubmitter to the new User object in userMap
+                                topologySubmitter = this.userMap.get(td.getTopologySubmitter());
+                                topologySubmitter.moveTopoFromPendingToAttempted(td);
                                 break;
                             }
                             continue;
                         } else if (result.getStatus() == SchedulingStatus.FAIL_INVALID_TOPOLOGY) {
+                            this.restoreCheckpointSchedulingState(schedulingState);
+                            //since state is restored need the update User topologySubmitter to the new User object in userMap
+                            topologySubmitter = this.userMap.get(td.getTopologySubmitter());
                             topologySubmitter.moveTopoFromPendingToInvalid(td, this.cluster);
-                            resetAssignments(assignmentCheckpoint);
                             break;
                         } else {
+                            this.restoreCheckpointSchedulingState(schedulingState);
+                            //since state is restored need the update User topologySubmitter to the new User object in userMap
+                            topologySubmitter = this.userMap.get(td.getTopologySubmitter());
                             topologySubmitter.moveTopoFromPendingToAttempted(td, this.cluster);
-                            resetAssignments(assignmentCheckpoint);
                             break;
                         }
                     }
                 } else {
                     LOG.warn("Scheduling results returned from topology {} is not vaild! Topology with be ignored.", td.getName());
+                    this.restoreCheckpointSchedulingState(schedulingState);
+                    //since state is restored need the update User topologySubmitter to the new User object in userMap
+                    topologySubmitter = this.userMap.get(td.getTopologySubmitter());
                     topologySubmitter.moveTopoFromPendingToInvalid(td, this.cluster);
-                    resetAssignments(assignmentCheckpoint);
                     break;
                 }
             }
@@ -215,6 +258,7 @@ public class ResourceAwareScheduler implements IScheduler {
     }
 
     private boolean mkAssignment(TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> schedulerAssignmentMap) {
+        LOG.info("making assignments for topology {}", td);
         if (schedulerAssignmentMap != null) {
             double requestedMemOnHeap = td.getTotalRequestedMemOnHeap();
             double requestedMemOffHeap = td.getTotalRequestedMemOffHeap();
@@ -272,69 +316,6 @@ public class ResourceAwareScheduler implements IScheduler {
         cluster.setSupervisorsResourcesMap(supervisors_resources);
     }
 
-
-//    private void scheduleTopology(TopologyDetails td) {
-//        ResourceAwareStrategy RAStrategy = new ResourceAwareStrategy(this.cluster, this.topologies);
-//        if (cluster.needsScheduling(td) && cluster.getUnassignedExecutors(td).size() > 0) {
-//            LOG.info("/********Scheduling topology {} from User {}************/", td.getName(), td.getTopologySubmitter());
-//            LOG.info("{}", this.userMap.get(td.getTopologySubmitter()).getDetailedInfo());
-//            LOG.info("{}", User.getResourcePoolAverageUtilizationForUsers(this.userMap.values()));
-//
-//            Map<WorkerSlot, Collection<ExecutorDetails>> schedulerAssignmentMap = RAStrategy.schedule(td);
-//
-//            double requestedMemOnHeap = td.getTotalRequestedMemOnHeap();
-//            double requestedMemOffHeap = td.getTotalRequestedMemOffHeap();
-//            double requestedCpu = td.getTotalRequestedCpu();
-//            double assignedMemOnHeap = 0.0;
-//            double assignedMemOffHeap = 0.0;
-//            double assignedCpu = 0.0;
-//
-//            if (schedulerAssignmentMap != null) {
-//                try {
-//                    Set<String> nodesUsed = new HashSet<String>();
-//                    int assignedWorkers = schedulerAssignmentMap.keySet().size();
-//                    for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> workerToTasksEntry : schedulerAssignmentMap.entrySet()) {
-//                        WorkerSlot targetSlot = workerToTasksEntry.getKey();
-//                        Collection<ExecutorDetails> execsNeedScheduling = workerToTasksEntry.getValue();
-//                        RAS_Node targetNode = RAStrategy.idToNode(targetSlot.getNodeId());
-//                        targetNode.assign(targetSlot, td, execsNeedScheduling, this.cluster);
-//                        LOG.debug("ASSIGNMENT    TOPOLOGY: {}  TASKS: {} To Node: {} on Slot: {}",
-//                                td.getName(), execsNeedScheduling, targetNode.getHostname(), targetSlot.getPort());
-//                        if (!nodesUsed.contains(targetNode.getId())) {
-//                            nodesUsed.add(targetNode.getId());
-//                        }
-//                        assignedMemOnHeap += targetSlot.getAllocatedMemOnHeap();
-//                        assignedMemOffHeap += targetSlot.getAllocatedMemOffHeap();
-//                        assignedCpu += targetSlot.getAllocatedCpu();
-//                    }
-//                    LOG.debug("Topology: {} assigned to {} nodes on {} workers", td.getId(), nodesUsed.size(), assignedWorkers);
-//                    this.cluster.setStatus(td.getId(), "Fully Scheduled");
-//                    this.getUser(td.getTopologySubmitter()).moveTopoFromPendingToRunning(td);
-//                    LOG.info("getNextUser: {}", this.getNextUser());
-//                } catch (IllegalStateException ex) {
-//                    LOG.error(ex.toString());
-//                    LOG.error("Unsuccessful in scheduling", td.getId());
-//                    this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
-//                    this.getUser(td.getTopologySubmitter()).moveTopoFromPendingToAttempted(td);
-//                }
-//            } else {
-//                LOG.error("Unsuccessful in scheduling {}", td.getId());
-//                this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
-//              //  this.evictTopology(td);
-//               // this.getUser(td.getTopologySubmitter()).moveTopoFromPendingToAttempted(td);
-//            }
-//            Double[] resources = {requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
-//                    assignedMemOnHeap, assignedMemOffHeap, assignedCpu};
-//            LOG.debug("setResources for {}: requested on-heap mem, off-heap mem, cpu: {} {} {} " +
-//                            "assigned on-heap mem, off-heap mem, cpu: {} {} {}",
-//                    td.getId(), requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
-//                    assignedMemOnHeap, assignedMemOffHeap, assignedCpu);
-//            this.cluster.setResources(td.getId(), resources);
-//        } else {
-//            LOG.warn("Topology {} already scheduled!", td.getName());
-//            this.cluster.setStatus(td.getId(), "Fully Scheduled");
-//        }
-//    }
     public User getUser(String user) {
         return this.userMap.get(user);
     }
@@ -346,23 +327,30 @@ public class ResourceAwareScheduler implements IScheduler {
     public User getNextUser() {
         Double least = Double.POSITIVE_INFINITY;
         User ret = null;
-        for(User user : this.userMap.values()) {
-            LOG.info("{}", user.getDetailedInfo());
+        for (User user : this.userMap.values()) {
+            LOG.info("getNextUser {}", user.getDetailedInfo());
             LOG.info("hasTopologyNeedSchedule: {}", user.hasTopologyNeedSchedule());
-            if(user.hasTopologyNeedSchedule()) {
+            if (user.hasTopologyNeedSchedule()) {
                 Double userResourcePoolAverageUtilization = user.getResourcePoolAverageUtilization();
+                if(ret!=null) {
+                    LOG.info("current: {}-{} compareUser: {}-{}", ret.getId(), least, user.getId(), userResourcePoolAverageUtilization);
+                    LOG.info("{} == {}: {}", least, userResourcePoolAverageUtilization, least == userResourcePoolAverageUtilization);
+                    LOG.info("{} == {}: {}", least, userResourcePoolAverageUtilization, (Math.abs(least - userResourcePoolAverageUtilization) < 0.0001));
+
+                }
                 if (least > userResourcePoolAverageUtilization) {
                     ret = user;
                     least = userResourcePoolAverageUtilization;
-                } else if (least == userResourcePoolAverageUtilization) {
-                    double currentCpuPercentage = ret.getCPUResourceGuaranteed()/this.cluster.getClusterTotalCPUResource();
-                    double currentMemoryPercentage = ret.getMemoryResourceGuaranteed()/this.cluster.getClusterTotalMemoryResource();
+                } else if (Math.abs(least - userResourcePoolAverageUtilization) < 0.0001) {
+                    double currentCpuPercentage = ret.getCPUResourceGuaranteed() / this.cluster.getClusterTotalCPUResource();
+                    double currentMemoryPercentage = ret.getMemoryResourceGuaranteed() / this.cluster.getClusterTotalMemoryResource();
                     double currentAvgPercentage = (currentCpuPercentage + currentMemoryPercentage) / 2.0;
 
-                    double userCpuPercentage = user.getCPUResourceGuaranteed()/this.cluster.getClusterTotalCPUResource();
-                    double userMemoryPercentage = user.getMemoryResourceGuaranteed()/this.cluster.getClusterTotalMemoryResource();
+                    double userCpuPercentage = user.getCPUResourceGuaranteed() / this.cluster.getClusterTotalCPUResource();
+                    double userMemoryPercentage = user.getMemoryResourceGuaranteed() / this.cluster.getClusterTotalMemoryResource();
                     double userAvgPercentage = (userCpuPercentage + userMemoryPercentage) / 2.0;
-                    if(userAvgPercentage > currentAvgPercentage) {
+                    LOG.info("current: {}-{} compareUser: {}-{}", ret.getId(), currentAvgPercentage, user.getId(), userAvgPercentage);
+                    if (userAvgPercentage > currentAvgPercentage) {
                         ret = user;
                         least = userResourcePoolAverageUtilization;
                     }
@@ -374,6 +362,7 @@ public class ResourceAwareScheduler implements IScheduler {
 
     /**
      * Intialize scheduling and running queues
+     *
      * @param topologies
      * @param cluster
      */
@@ -384,18 +373,16 @@ public class ResourceAwareScheduler implements IScheduler {
         LOG.info("userResourcePools: {}", userResourcePools);
 
         for (TopologyDetails td : topologies.getTopologies()) {
-            LOG.info("topology: {} from {}", td.getName(), td.getTopologySubmitter());
             String topologySubmitter = td.getTopologySubmitter();
-            if(topologySubmitter == null) {
+            if (topologySubmitter == null) {
                 LOG.warn("Topology {} submitted by anonymous user", td.getName());
                 topologySubmitter = "anonymous";
             }
-            if(!this.userMap.containsKey(topologySubmitter)) {
+            if (!this.userMap.containsKey(topologySubmitter)) {
                 this.userMap.put(topologySubmitter, new User(topologySubmitter, userResourcePools.get(topologySubmitter)));
             }
-            if(cluster.getUnassignedExecutors(td).size() >= td.getExecutors().size()) {
+            if (cluster.getUnassignedExecutors(td).size() >= td.getExecutors().size()) {
                 this.userMap.get(topologySubmitter).addTopologyToPendingQueue(td, cluster);
-                LOG.info(this.userMap.get(topologySubmitter).getDetailedInfo());
             } else {
                 this.userMap.get(topologySubmitter).addTopologyToRunningQueue(td, cluster);
             }
@@ -411,35 +398,62 @@ public class ResourceAwareScheduler implements IScheduler {
 
     /**
      * Get resource guarantee configs
+     *
      * @return
      */
     private Map<String, Map<String, Double>> getUserResourcePools() {
         Object raw = this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS);
-        Map<String, Map<String, Double>> ret =  (Map<String, Map<String, Double>>)this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS);
+        Map<String, Map<String, Double>> ret = new HashMap<String, Map<String, Double>>();
 
-        if (raw == null) {
-            ret = new HashMap<String, Map<String, Double>>();
-        } else {
-            for(Map.Entry<String, Map<String, Number>> UserPoolEntry : ((Map<String, Map<String, Number>>) raw).entrySet()) {
+        if (raw != null) {
+            for (Map.Entry<String, Map<String, Number>> UserPoolEntry : ((Map<String, Map<String, Number>>) raw).entrySet()) {
                 String user = UserPoolEntry.getKey();
                 ret.put(user, new HashMap<String, Double>());
-                for(Map.Entry<String, Number> resourceEntry : UserPoolEntry.getValue().entrySet()) {
+                for (Map.Entry<String, Number> resourceEntry : UserPoolEntry.getValue().entrySet()) {
                     ret.get(user).put(resourceEntry.getKey(), resourceEntry.getValue().doubleValue());
                 }
             }
         }
 
         Map fromFile = Utils.findAndReadConfigFile("user-resource-pools.yaml", false);
-        Map<String, Map<String, Number>>tmp = (Map<String, Map<String, Number>>)fromFile.get(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS);
+        Map<String, Map<String, Number>> tmp = (Map<String, Map<String, Number>>) fromFile.get(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS);
         if (tmp != null) {
-            for(Map.Entry<String, Map<String, Number>> UserPoolEntry : tmp.entrySet()) {
+            for (Map.Entry<String, Map<String, Number>> UserPoolEntry : tmp.entrySet()) {
                 String user = UserPoolEntry.getKey();
                 ret.put(user, new HashMap<String, Double>());
-                for(Map.Entry<String, Number> resourceEntry : UserPoolEntry.getValue().entrySet()) {
+                for (Map.Entry<String, Number> resourceEntry : UserPoolEntry.getValue().entrySet()) {
                     ret.get(user).put(resourceEntry.getKey(), resourceEntry.getValue().doubleValue());
                 }
             }
         }
         return ret;
     }
+
+    private SchedulingState checkpointSchedulingState() {
+        LOG.info("checkpointing scheduling state...");
+        LOG.info("/*********Checkpoint************/");
+        for (User user : this.getUserMap().values()) {
+            LOG.info(user.getDetailedInfo());
+        }
+        LOG.info("/*********End************/");
+        return new SchedulingState(this.userMap, this.cluster, this.topologies, this.nodes, this.conf);
+    }
+
+    private void restoreCheckpointSchedulingState(SchedulingState schedulingState) {
+        LOG.info("restoring scheduling state...");
+        LOG.info("/*********Before************/");
+        for (User user : this.getUserMap().values()) {
+            LOG.info(user.getDetailedInfo());
+        }
+        this.cluster = schedulingState.cluster;
+        this.topologies = schedulingState.topologies;
+        this.conf = schedulingState.conf;
+        this.userMap = schedulingState.userMap;
+        this.nodes = schedulingState.nodes;
+        LOG.info("/*********After************/");
+        for (User user : this.getUserMap().values()) {
+            LOG.info(user.getDetailedInfo());
+        }
+        LOG.info("/*********End************/");
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3c864c/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java
index 8e11384..02d48e1 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java
@@ -181,4 +181,13 @@ public class ResourceUtils {
         }
         return str.toString();
     }
+
+    public static String printScheduling(RAS_Nodes nodes) {
+        String ret="";
+        for (RAS_Node node : nodes.getNodes()) {
+            ret += "Node: " + node.getHostname() + "\n";
+            ret += "-> " + node.getTopoIdTousedSlots() + "\n";
+        }
+        return ret;
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3c864c/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
index 068db54..77a1dff 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
@@ -35,15 +35,15 @@ import java.util.TreeSet;
 public class User {
     private String userId;
     //Topologies yet to be scheduled sorted by priority for each user
-    private Set<TopologyDetails> pendingQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
+    private TreeSet<TopologyDetails> pendingQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
 
     //Topologies yet to be scheduled sorted by priority for each user
-    private Set<TopologyDetails> runningQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
+    private TreeSet<TopologyDetails> runningQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
 
     //Topologies that was attempted to be scheduled but wasn't successull
-    private Set<TopologyDetails> attemptedQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
+    private TreeSet<TopologyDetails> attemptedQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
 
-    private Set<TopologyDetails> invalidQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
+    private TreeSet<TopologyDetails> invalidQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
 
     private Map<String, Double> resourcePool = new HashMap<String, Double>();
 
@@ -60,6 +60,23 @@ public class User {
         }
     }
 
+    public User getCopy() {
+        User newUser = new User(this.userId, this.resourcePool);
+        for(TopologyDetails topo :  this.pendingQueue) {
+            newUser.addTopologyToPendingQueue(topo);
+        }
+        for(TopologyDetails topo :  this.runningQueue) {
+            newUser.addTopologyToRunningQueue(topo);
+        }
+        for(TopologyDetails topo :  this.attemptedQueue) {
+            newUser.addTopologyToAttemptedQueue(topo);
+        }
+        for(TopologyDetails topo :  this.invalidQueue) {
+            newUser.addTopologyToInvalidQueue(topo);
+        }
+        return newUser;
+    }
+
     public String getId() {
         return this.userId;
     }
@@ -92,6 +109,14 @@ public class User {
         return ret;
     }
 
+    public void addTopologyToAttemptedQueue(TopologyDetails topo) {
+        this.attemptedQueue.add(topo);
+    }
+
+    public void addTopologyToInvalidQueue(TopologyDetails topo) {
+        this.invalidQueue.add(topo);
+    }
+
     public Set<TopologyDetails> getTopologiesRunning() {
         TreeSet<TopologyDetails> ret = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
         ret.addAll(this.runningQueue);
@@ -104,6 +129,11 @@ public class User {
         return ret;
     }
 
+    public Set<TopologyDetails> getTopologiesInvalid() {
+        TreeSet<TopologyDetails> ret = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
+        ret.addAll(this.invalidQueue);
+        return ret;
+    }
     public Map<String, Number> getResourcePool() {
         if (this.resourcePool != null) {
             return new HashMap<String, Number>(this.resourcePool);
@@ -160,17 +190,12 @@ public class User {
 
 
     private void moveTopology(TopologyDetails topo, Set<TopologyDetails> src, String srcName, Set<TopologyDetails> dest, String destName) {
-        LOG.info("{} queue: {}", srcName, src);
-        LOG.info("{} queue: {}", destName, dest);
+        LOG.info("For User {} Moving topo {} from {} to {}", this.userId, topo.getName(), srcName, destName);
         if (topo == null) {
             return;
         }
         if (!src.contains(topo)) {
             LOG.warn("Topo {} not in User: {} {} queue!", topo.getName(), this.userId, srcName);
-            LOG.info("topo {}-{}-{}", topo.getName(), topo.getId(), topo.hashCode());
-            for (TopologyDetails t : src) {
-                LOG.info("queue entry: {}-{}-{}", t.getName(), t.getId(), t.hashCode());
-            }
             return;
         }
         if (dest.contains(topo)) {
@@ -179,6 +204,8 @@ public class User {
         }
         src.remove(topo);
         dest.add(topo);
+        LOG.info("SRC: {}", src);
+        LOG.info("DEST: {}", dest);
     }
 
 
@@ -244,14 +271,15 @@ public class User {
     }
 
     public boolean hasTopologyNeedSchedule() {
-        return (!this.pendingQueue.isEmpty() && (this.pendingQueue.size() - this.attemptedQueue.size()) > 0);
+        //return (!this.pendingQueue.isEmpty() && (this.pendingQueue.size() - this.attemptedQueue.size()) > 0);
+        return (!this.pendingQueue.isEmpty());
     }
 
     public TopologyDetails getRunningTopologyWithLowestPriority() {
         if (this.runningQueue.isEmpty()) {
             return null;
         }
-        return this.runningQueue.iterator().next();
+        return this.runningQueue.last();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3c864c/storm-core/test/jvm/backtype/storm/scheduler/resource/Experiment.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/Experiment.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/Experiment.java
index bea41ff..3ff0af1 100644
--- a/storm-core/test/jvm/backtype/storm/scheduler/resource/Experiment.java
+++ b/storm-core/test/jvm/backtype/storm/scheduler/resource/Experiment.java
@@ -29,6 +29,8 @@ import backtype.storm.utils.Time;
 import backtype.storm.utils.Utils;
 import org.junit.Test;
 import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -37,120 +39,61 @@ import java.util.Map;
  * Created by jerrypeng on 11/11/15.
  */
 public class Experiment {
-    @Test
-    public void TestMultipleUsers() {
-//        INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
-//        Map<String, Number> resourceMap = new HashMap<String, Number>();
-//        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 1000.0);
-//        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1024.0 * 10);
-//        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(5, 4, resourceMap);
-//        Config config = new Config();
-//        config.putAll(Utils.readDefaultConfig());
-//        Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
-//        resourceUserPool.put("jerry", new HashMap<String, Number>());
-//        resourceUserPool.get("jerry").put("cpu", 1000);
-//        resourceUserPool.get("jerry").put("memory", 8192.0);
-//
-//        resourceUserPool.put("bobby", new HashMap<String, Number>());
-//        resourceUserPool.get("bobby").put("cpu", 10000.0);
-//        resourceUserPool.get("bobby").put("memory", 32768);
-//
-//        resourceUserPool.put("derek", new HashMap<String, Number>());
-//        resourceUserPool.get("derek").put("cpu", 5000.0);
-//        resourceUserPool.get("derek").put("memory", 16384.0);
-//
-//        config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
-//        Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), config);
-//
-//        config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
-//
-//        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
-//        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
-//        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30);
-//        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
-//        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30);
-//
-//        config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
-//
-//        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
-//        TopologyDetails topo7 = TestUtilsForResourceAwareScheduler.getTopology("topo-7", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
-//        TopologyDetails topo8 = TestUtilsForResourceAwareScheduler.getTopology("topo-8", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30);
-//        TopologyDetails topo9 = TestUtilsForResourceAwareScheduler.getTopology("topo-9", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
-//        TopologyDetails topo10 = TestUtilsForResourceAwareScheduler.getTopology("topo-10", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30);
-//
-//        config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
-//
-//        TopologyDetails topo11 = TestUtilsForResourceAwareScheduler.getTopology("topo-11", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
-//        TopologyDetails topo12 = TestUtilsForResourceAwareScheduler.getTopology("topo-12", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
-//        TopologyDetails topo13 = TestUtilsForResourceAwareScheduler.getTopology("topo-13", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30);
-//        TopologyDetails topo14 = TestUtilsForResourceAwareScheduler.getTopology("topo-14", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
-//        TopologyDetails topo15 = TestUtilsForResourceAwareScheduler.getTopology("topo-15", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30);
-//
-//
-//        Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
-//        topoMap.put(topo1.getId(), topo1);
-//        topoMap.put(topo2.getId(), topo2);
-//        topoMap.put(topo3.getId(), topo3);
-//        topoMap.put(topo4.getId(), topo4);
-//        topoMap.put(topo5.getId(), topo5);
-//        topoMap.put(topo6.getId(), topo6);
-//        topoMap.put(topo7.getId(), topo7);
-//        topoMap.put(topo8.getId(), topo8);
-//        topoMap.put(topo9.getId(), topo9);
-//        topoMap.put(topo10.getId(), topo10);
-//        topoMap.put(topo11.getId(), topo11);
-//        topoMap.put(topo12.getId(), topo12);
-//        topoMap.put(topo13.getId(), topo13);
-//        topoMap.put(topo14.getId(), topo14);
-//        topoMap.put(topo15.getId(), topo15);
-//
-//        Topologies topologies = new Topologies(topoMap);
-//
-//        ResourceAwareScheduler rs = new ResourceAwareScheduler();
-//
-//        rs.prepare(config);
-//        rs.schedule(topologies, cluster);
-//
-//        for(TopologyDetails topo : topoMap.values()) {
-//            Assert.assertEquals(cluster.getStatusMap().get(topo.getId()), "Fully Scheduled");
-//        }
-//
-//        for(User user : rs.getUserMap().values()) {
-//            Assert.assertEquals(user.getTopologiesPending().size(), 0);
-//            Assert.assertEquals(user.getTopologiesRunning().size(), 5);
-//        }
 
+    private static final Logger LOG = LoggerFactory.getLogger(Experiment.class);
+
+    /**
+     * Eviction order:
+     * topo-3: since user bobby don't have any resource guarantees and topo-3 is the lowest priority for user bobby
+     * topo-2: since user bobby don't have any resource guarantees and topo-2 is the next lowest priority for user bobby
+     * topo-5: since user derek has exceeded his resource guarantee while user jerry has not.  topo-5 and topo-4 has the same priority
+     * but topo-4 was submitted earlier thus we choose that one to evict
+     */
+    @Test
+    public void TestEvictMultipleTopologiesFromMultipleUsersInCorrectOrder() {
         INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
         Map<String, Number> resourceMap = new HashMap<String, Number>();
-        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 200.0);
-        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1024.0 * 10);
-        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(1, 4, resourceMap);
+        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 100.0);
+        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1000.0);
+        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(4, 4, resourceMap);
         Config config = new Config();
         config.putAll(Utils.readDefaultConfig());
+        config.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 100.0);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 500);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 500);
         Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
         resourceUserPool.put("jerry", new HashMap<String, Number>());
-        resourceUserPool.get("jerry").put("cpu", 1000);
-        resourceUserPool.get("jerry").put("memory", 8192.0);
-
-        resourceUserPool.put("bobby", new HashMap<String, Number>());
-        resourceUserPool.get("bobby").put("cpu", 10000.0);
-        resourceUserPool.get("bobby").put("memory", 32768);
+        resourceUserPool.get("jerry").put("cpu", 300.0);
+        resourceUserPool.get("jerry").put("memory", 3000.0);
 
         resourceUserPool.put("derek", new HashMap<String, Number>());
-        resourceUserPool.get("derek").put("cpu", 5000.0);
-        resourceUserPool.get("derek").put("memory", 16384.0);
+        resourceUserPool.get("derek").put("cpu", 100.0);
+        resourceUserPool.get("derek").put("memory", 1000.0);
 
         config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
         Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), config);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
 
-        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
-        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo7 = TestUtilsForResourceAwareScheduler.getTopology("topo-7", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
+
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
+
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 15, 30);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
-        topoMap.put(topo1.getId(), topo1);
         topoMap.put(topo2.getId(), topo2);
+        topoMap.put(topo3.getId(), topo3);
+        topoMap.put(topo4.getId(), topo4);
+        topoMap.put(topo5.getId(), topo5);
 
         Topologies topologies = new Topologies(topoMap);
 
@@ -159,16 +102,120 @@ public class Experiment {
         rs.prepare(config);
         rs.schedule(topologies, cluster);
 
-        int fullyScheduled = 0;
-        for (TopologyDetails topo : topoMap.values()) {
-            if(cluster.getStatusMap().get(topo.getId()).equals("Fully Scheduled")) {
-                fullyScheduled++;
-            }
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("bobby").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("bobby").getTopologiesAttempted().size());
+
+        //user jerry submits another topology
+        topoMap.put(topo1.getId(), topo1);
+        topologies = new Topologies(topoMap);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesAttempted()) {
+            Assert.assertFalse("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of attempted topologies", 1, rs.getUser("bobby").getTopologiesAttempted().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("bobby").getTopologiesRunning().size());
+        Assert.assertEquals("correct topology to evict", rs.getUser("bobby").getTopologiesAttempted().iterator().next().getName(), "topo-3");
+
+        topoMap.put(topo6.getId(), topo6);
+        topologies = new Topologies(topoMap);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
-        Assert.assertEquals("# of Fully scheduled", 1, fullyScheduled);
-        Assert.assertEquals("# of topologies schedule attempted", 1, rs.getUser("jerry").getTopologiesAttempted().size());
-        Assert.assertEquals("# of topologies running", 1, rs.getUser("jerry").getTopologiesRunning().size());
-        Assert.assertEquals("# of topologies schedule pending", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesAttempted()) {
+            Assert.assertFalse(TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of attempted topologies", 2, rs.getUser("bobby").getTopologiesAttempted().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+        Assert.assertEquals("# of running topologies", 0, rs.getUser("bobby").getTopologiesRunning().size());
+
+        Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-2", rs.getUser("bobby").getTopologiesAttempted()) != null);
+        Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-3", rs.getUser("bobby").getTopologiesAttempted()) != null);
+
+        topoMap.put(topo7.getId(), topo7);
+        topologies = new Topologies(topoMap);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 3, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesAttempted()) {
+            Assert.assertFalse("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 1, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+        Assert.assertEquals("correct topology to evict", rs.getUser("derek").getTopologiesAttempted().iterator().next().getName(), "topo-4");
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesAttempted()) {
+            Assert.assertFalse("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of attempted topologies", 2, rs.getUser("bobby").getTopologiesAttempted().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+        Assert.assertEquals("# of running topologies", 0, rs.getUser("bobby").getTopologiesRunning().size());
+
+        Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-2", rs.getUser("bobby").getTopologiesAttempted()) != null);
+        Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-3", rs.getUser("bobby").getTopologiesAttempted()) != null);
     }
 
 


[03/23] storm git commit: adding unit tests for STORM-898

Posted by da...@apache.org.
adding unit tests for STORM-898


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

Branch: refs/heads/master
Commit: 3f55feef43602beefebdb6ae20c64bceff2dbecb
Parents: 97b2c9a
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Thu Nov 12 09:48:07 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Fri Dec 4 13:06:45 2015 -0600

----------------------------------------------------------------------
 conf/defaults.yaml                              |   1 +
 .../jvm/backtype/storm/scheduler/Cluster.java   |  17 ++
 .../storm/scheduler/SupervisorDetails.java      |  12 +-
 .../backtype/storm/scheduler/Topologies.java    |   9 +
 .../storm/scheduler/TopologyDetails.java        |  43 ++-
 .../storm/scheduler/resource/RAS_Node.java      | 163 ++--------
 .../storm/scheduler/resource/RAS_Nodes.java     | 149 +++++++++
 .../resource/ResourceAwareScheduler.java        | 285 ++++++++++++++---
 .../scheduler/resource/SchedulingResult.java    | 111 +++++++
 .../scheduler/resource/SchedulingStatus.java    |  40 +++
 .../backtype/storm/scheduler/resource/User.java | 122 +++++++-
 .../resource/strategies/IStrategy.java          |   3 +-
 .../strategies/ResourceAwareStrategy.java       |  17 +-
 .../scheduler/resource_aware_scheduler_test.clj | 126 +++++---
 .../jvm/backtype/storm/TestConfigValidate.java  |  58 ++++
 .../storm/scheduler/resource/Experiment.java    | 175 +++++++++++
 .../resource/TestResourceAwareScheduler.java    | 305 +++++++++++++++++++
 .../storm/scheduler/resource/TestUser.java      | 111 +++++++
 .../TestUtilsForResourceAwareScheduler.java     | 265 ++++++++++++++++
 19 files changed, 1742 insertions(+), 270 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index aeda11c..2a99ba6 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -249,6 +249,7 @@ topology.disruptor.wait.timeout.millis: 1000
 topology.disruptor.batch.size: 100
 topology.disruptor.batch.timeout.millis: 1
 topology.disable.loadaware: false
+topology.priority: 30
 
 # Configs for Resource Aware Scheduler
 topology.component.resources.onheap.memory.mb: 128.0

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
index f4d12d8..7f16b86 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
@@ -379,6 +379,16 @@ public class Cluster {
     }
 
     /**
+     * get slots used by a topology
+     */
+    public Collection<WorkerSlot> getUsedSlotsByTopologyId(String topologyId) {
+        if (!this.assignments.containsKey(topologyId)) {
+            return null;
+        }
+        return this.assignments.get(topologyId).getSlots();
+    }
+
+    /**
      * Get a specific supervisor with the <code>nodeId</code>
      */
     public SupervisorDetails getSupervisorById(String nodeId) {
@@ -429,6 +439,13 @@ public class Cluster {
         return ret;
     }
 
+    public void setAssignments(Map<String, SchedulerAssignment> assignments) {
+        this.assignments = new HashMap<String, SchedulerAssignmentImpl>();
+        for(Map.Entry<String, SchedulerAssignmentImpl> entry : this.assignments.entrySet()) {
+            this.assignments.put(entry.getKey(), new SchedulerAssignmentImpl(entry.getValue().getTopologyId(), entry.getValue().getExecutorToSlot()));
+        }
+    }
+
     /**
      * Get all the supervisors.
      */

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
index a748e11..d93252f 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
@@ -119,10 +119,18 @@ public class SupervisorDetails {
     }
 
     public Double getTotalMemory() {
-        return getTotalResource(Config.SUPERVISOR_MEMORY_CAPACITY_MB);
+        Double totalMemory = getTotalResource(Config.SUPERVISOR_MEMORY_CAPACITY_MB);
+        if(totalMemory == null) {
+            throw new IllegalStateException("default value for supervisor.memory.capacity.mb is not set!");
+        }
+        return totalMemory;
     }
 
     public Double getTotalCPU() {
-        return getTotalResource(Config.SUPERVISOR_CPU_CAPACITY);
+        Double totalCPU = getTotalResource(Config.SUPERVISOR_CPU_CAPACITY);
+        if(totalCPU == null) {
+            throw new IllegalStateException("default value for supervisor.cpu.capacity is not set!");
+        }
+        return totalCPU;
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
index 443bf3f..0b4c0ca 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
@@ -67,4 +67,13 @@ public class Topologies {
         }
         return _allComponents;
     }
+
+    @Override
+    public String toString() {
+        String ret = "Topologies:\n";
+        for(TopologyDetails td : this.getTopologies()) {
+            ret += td.toString() + "\n";
+        }
+        return ret;
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
index ac515b7..3931b43 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
@@ -63,17 +63,22 @@ public class TopologyDetails {
     }
 
     public TopologyDetails(String topologyId, Map topologyConf, StormTopology topology,
-                           int numWorkers, Map<ExecutorDetails, String> executorToComponents, int launchTime) {
+                           int numWorkers, Map<ExecutorDetails, String> executorToComponents) {
         this(topologyId, topologyConf, topology, numWorkers);
         this.executorToComponent = new HashMap<>(0);
         if (executorToComponents != null) {
             this.executorToComponent.putAll(executorToComponents);
         }
-        this.launchTime = launchTime;
         this.initResourceList();
         this.initConfigs();
     }
 
+    public TopologyDetails(String topologyId, Map topologyConf, StormTopology topology,
+                           int numWorkers, Map<ExecutorDetails, String> executorToComponents, int launchTime) {
+        this(topologyId, topologyConf, topology, numWorkers, executorToComponents);
+        this.launchTime = launchTime;
+    }
+
     public String getId() {
         return topologyId;
     }
@@ -410,13 +415,24 @@ public class TopologyDetails {
      * Add default resource requirements for a executor
      */
     public void addDefaultResforExec(ExecutorDetails exec) {
+
+        Double topologyComponentCpuPcorePercent = Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT), null);
+        if(topologyComponentCpuPcorePercent == null) {
+            LOG.warn("default value for topology.component.cpu.pcore.percent needs to be set!");
+        }
+        Double topologyComponentResourcesOffheapMemoryMb =  Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB), null);
+        if(topologyComponentResourcesOffheapMemoryMb == null) {
+            LOG.warn("default value for topology.component.resources.offheap.memory.mb needs to be set!");
+        }
+        Double topologyComponentResourcesOnheapMemoryMb = Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB), null);
+        if(topologyComponentResourcesOnheapMemoryMb == null) {
+            LOG.warn("default value for topology.component.resources.onheap.memory.mb needs to be set!");
+        }
+
         Map<String, Double> defaultResourceList = new HashMap<>();
-        defaultResourceList.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT,
-                        Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT), null));
-        defaultResourceList.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB,
-                        Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB), null));
-        defaultResourceList.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB,
-                        Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB), null));
+        defaultResourceList.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, topologyComponentCpuPcorePercent);
+        defaultResourceList.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, topologyComponentResourcesOffheapMemoryMb);
+        defaultResourceList.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, topologyComponentResourcesOnheapMemoryMb);
         LOG.debug("Scheduling Executor: {} with memory requirement as onHeap: {} - offHeap: {} " +
                         "and CPU requirement: {}",
                 exec, topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB),
@@ -426,12 +442,17 @@ public class TopologyDetails {
     }
 
     /**
-     * initializes the scheduler member variable by extracting what scheduler
-     * this topology is going to use from topologyConf
+     * initializes member variables
      */
     private void initConfigs() {
         this.topologyWorkerMaxHeapSize = Utils.getDouble(this.topologyConf.get(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB), null);
+        if(this.topologyWorkerMaxHeapSize == null) {
+            LOG.warn("default value for topology.worker.max.heap.size.mb needs to be set!");
+        }
         this.topologyPriority = Utils.getInt(this.topologyConf.get(Config.TOPOLOGY_PRIORITY), null);
+        if(this.topologyPriority == null) {
+            LOG.warn("default value for topology.priority needs to be set!");
+        }
     }
 
     /**
@@ -459,7 +480,7 @@ public class TopologyDetails {
 
     @Override
     public String toString() {
-        return "Name: " + this.getName() + " Priority: " + this.getTopologyPriority()
+        return "Name: " + this.getName() + " id: " + this.getId() + " Priority: " + this.getTopologyPriority()
                 + " Uptime: " + this.getUpTime() + " CPU: " + this.getTotalRequestedCpu()
                 + " Memory: " + (this.getTotalRequestedMemOffHeap() + this.getTotalRequestedMemOnHeap());
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
index 1f2e795..b0bcc8a 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
@@ -21,6 +21,7 @@ package backtype.storm.scheduler.resource;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -53,14 +54,11 @@ public class RAS_Node {
     private Double _availMemory;
     private Double _availCPU;
     private List<WorkerSlot> _slots;
-    private List<ExecutorDetails> _execs;
-    private Map<WorkerSlot, List<ExecutorDetails>> _slotToExecs;
+    private Cluster _cluster;
 
     public RAS_Node(String nodeId, Set<Integer> allPorts, boolean isAlive,
-                    SupervisorDetails sup) {
+                    SupervisorDetails sup, Cluster cluster) {
         _slots = new ArrayList<WorkerSlot>();
-        _execs = new ArrayList<ExecutorDetails>();
-        _slotToExecs = new HashMap<WorkerSlot, List<ExecutorDetails>>();
         _nodeId = nodeId;
         _isAlive = isAlive;
         if (_isAlive && allPorts != null) {
@@ -72,10 +70,8 @@ public class RAS_Node {
             _availMemory = this.getTotalMemoryResources();
             _availCPU = this.getTotalCpuResources();
             _slots.addAll(_freeSlots);
-            for (WorkerSlot ws : _slots) {
-                _slotToExecs.put(ws, new ArrayList<ExecutorDetails>());
-            }
         }
+        this._cluster = cluster;
     }
 
     public String getId() {
@@ -90,6 +86,14 @@ public class RAS_Node {
         return _freeSlots;
     }
 
+    public Collection<WorkerSlot> getUsedSlots() {
+        Collection<WorkerSlot> ret = new LinkedList<WorkerSlot>();
+        for(Collection<WorkerSlot> workers : _topIdToUsedSlots.values()) {
+            ret.addAll(workers);
+        }
+        return ret;
+    }
+
     public boolean isAlive() {
         return _isAlive;
     }
@@ -138,7 +142,7 @@ public class RAS_Node {
         }
     }
 
-    private void addOrphanedSlot(WorkerSlot ws) {
+     void addOrphanedSlot(WorkerSlot ws) {
         if (_isAlive) {
             throw new IllegalArgumentException("Orphaned Slots " +
                     "only are allowed on dead nodes.");
@@ -153,7 +157,6 @@ public class RAS_Node {
             }
         }
         _freeSlots.add(ws);
-        _slotToExecs.put(ws, new ArrayList<ExecutorDetails>());
     }
 
     boolean assignInternal(WorkerSlot ws, String topId, boolean dontThrow) {
@@ -175,14 +178,13 @@ public class RAS_Node {
 
     /**
      * Free all slots on this node.  This will update the Cluster too.
-     * @param cluster the cluster to be updated
      */
-    public void freeAllSlots(Cluster cluster) {
+    public void freeAllSlots() {
         if (!_isAlive) {
             LOG.warn("Freeing all slots on a dead node {} ", _nodeId);
         }
         for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
-            cluster.freeSlots(entry.getValue());
+            _cluster.freeSlots(entry.getValue());
             if (_isAlive) {
                 _freeSlots.addAll(entry.getValue());
             }
@@ -193,14 +195,13 @@ public class RAS_Node {
     /**
      * Frees a single slot in this node
      * @param ws the slot to free
-     * @param cluster the cluster to update
      */
-    public void free(WorkerSlot ws, Cluster cluster) {
+    public void free(WorkerSlot ws) {
         if (_freeSlots.contains(ws)) return;
         for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
             Set<WorkerSlot> slots = entry.getValue();
             if (slots.remove(ws)) {
-                cluster.freeSlot(ws);
+                _cluster.freeSlot(ws);
                 if (_isAlive) {
                     _freeSlots.add(ws);
                 }
@@ -214,15 +215,14 @@ public class RAS_Node {
     /**
      * Frees all the slots for a topology.
      * @param topId the topology to free slots for
-     * @param cluster the cluster to update
      */
-    public void freeTopology(String topId, Cluster cluster) {
+    public void freeTopology(String topId) {
         Set<WorkerSlot> slots = _topIdToUsedSlots.get(topId);
         if (slots == null || slots.isEmpty()) {
             return;
         }
         for (WorkerSlot ws : slots) {
-            cluster.freeSlot(ws);
+            _cluster.freeSlot(ws);
             if (_isAlive) {
                 _freeSlots.add(ws);
             }
@@ -257,8 +257,7 @@ public class RAS_Node {
         slot.allocateResource(onHeapMem, offHeapMem, cpu);
     }
 
-    public void assign(WorkerSlot target, TopologyDetails td, Collection<ExecutorDetails> executors,
-                       Cluster cluster) {
+    public void assign(WorkerSlot target, TopologyDetails td, Collection<ExecutorDetails> executors) {
         if (!_isAlive) {
             throw new IllegalStateException("Trying to adding to a dead node " + _nodeId);
         }
@@ -276,7 +275,7 @@ public class RAS_Node {
             throw new IllegalStateException("Trying to assign already used slot" + target.getPort() + "on node " + _nodeId);
         } else {
             allocateResourceToSlot(td, executors, target);
-            cluster.assign(target, td.getId(), executors);
+            _cluster.assign(target, td.getId(), executors);
             assignInternal(target, td.getId(), false);
         }
     }
@@ -286,11 +285,9 @@ public class RAS_Node {
      * This will update the cluster too.
      * @param td the TopologyDetails to assign a free slot to.
      * @param executors the executors to run in that slot.
-     * @param cluster the cluster to be updated
      */
-    public void assign(TopologyDetails td, Collection<ExecutorDetails> executors,
-                       Cluster cluster) {
-        this.assign(null, td, executors, cluster);
+    public void assign(TopologyDetails td, Collection<ExecutorDetails> executors) {
+        this.assign(null, td, executors);
     }
 
     @Override
@@ -347,70 +344,8 @@ public class RAS_Node {
         return total;
     }
 
-    public static Map<String, RAS_Node> getAllNodesFrom(Cluster cluster, Topologies topologies) {
-        Map<String, RAS_Node> nodeIdToNode = new HashMap<String, RAS_Node>();
-        for (SupervisorDetails sup : cluster.getSupervisors().values()) {
-            //Node ID and supervisor ID are the same.
-            String id = sup.getId();
-            boolean isAlive = !cluster.isBlackListed(id);
-            LOG.debug("Found a {} Node {} {}",
-                    isAlive ? "living" : "dead", id, sup.getAllPorts());
-            LOG.debug("resources_mem: {}, resources_CPU: {}", sup.getTotalMemory(), sup.getTotalCPU());
-            nodeIdToNode.put(sup.getId(), new RAS_Node(id, sup.getAllPorts(), isAlive, sup));
-        }
-        for (Entry<String, SchedulerAssignment> entry : cluster.getAssignments().entrySet()) {
-            String topId = entry.getValue().getTopologyId();
-            for (WorkerSlot workerSlot : entry.getValue().getSlots()) {
-                String id = workerSlot.getNodeId();
-                RAS_Node node = nodeIdToNode.get(id);
-                if (node == null) {
-                    LOG.info("Found an assigned slot on a dead supervisor {} with executors {}",
-                            workerSlot, getExecutors(workerSlot, cluster));
-                    node = new RAS_Node(id, null, false, null);
-                    nodeIdToNode.put(id, node);
-                }
-                if (!node.isAlive()) {
-                    //The supervisor on the node down so add an orphaned slot to hold the unsupervised worker
-                    node.addOrphanedSlot(workerSlot);
-                }
-                if (node.assignInternal(workerSlot, topId, true)) {
-                    LOG.warn("Bad scheduling state, " + workerSlot + " assigned multiple workers, unassigning everything...");
-                    node.free(workerSlot, cluster);
-                }
-            }
-        }
-        RAS_Node.updateAvailableResources(cluster, topologies, nodeIdToNode);
-
-        for (Map.Entry<String, SchedulerAssignment> entry : cluster
-                .getAssignments().entrySet()) {
-            for (Map.Entry<ExecutorDetails, WorkerSlot> exec : entry.getValue()
-                    .getExecutorToSlot().entrySet()) {
-                ExecutorDetails ed = exec.getKey();
-                WorkerSlot ws = exec.getValue();
-                String node_id = ws.getNodeId();
-                if (nodeIdToNode.containsKey(node_id)) {
-                    RAS_Node node = nodeIdToNode.get(node_id);
-                    if (node._slotToExecs.containsKey(ws)) {
-                        node._slotToExecs.get(ws).add(ed);
-                        node._execs.add(ed);
-                    } else {
-                        LOG.info(
-                                "ERROR: should have node {} should have worker: {}",
-                                node_id, ed);
-                        return null;
-                    }
-                } else {
-                    LOG.info("ERROR: should have node {}", node_id);
-                    return null;
-                }
-            }
-        }
-        return nodeIdToNode;
-    }
-
     //This function is only used for logging information
-    private static Collection<ExecutorDetails> getExecutors(WorkerSlot ws,
-                                                            Cluster cluster) {
+    public static Collection<ExecutorDetails> getExecutors(WorkerSlot ws, Cluster cluster) {
         Collection<ExecutorDetails> retList = new ArrayList<ExecutorDetails>();
         for (Entry<String, SchedulerAssignment> entry : cluster.getAssignments()
                 .entrySet()) {
@@ -430,56 +365,6 @@ public class RAS_Node {
     }
 
     /**
-     * updates the available resources for every node in a cluster
-     * by recalculating memory requirements.
-     * @param cluster the cluster used in this calculation
-     * @param topologies container of all topologies
-     * @param nodeIdToNode a map between node id and node
-     */
-    private static void updateAvailableResources(Cluster cluster,
-                                                 Topologies topologies,
-                                                 Map<String, RAS_Node> nodeIdToNode) {
-        //recompute memory
-        if (cluster.getAssignments().size() > 0) {
-            for (Entry<String, SchedulerAssignment> entry : cluster.getAssignments()
-                    .entrySet()) {
-                Map<ExecutorDetails, WorkerSlot> executorToSlot = entry.getValue()
-                        .getExecutorToSlot();
-                Map<ExecutorDetails, Double> topoMemoryResourceList = topologies.getById(entry.getKey()).getTotalMemoryResourceList();
-
-                if (topoMemoryResourceList == null || topoMemoryResourceList.size() == 0) {
-                    continue;
-                }
-                for (Map.Entry<ExecutorDetails, WorkerSlot> execToSlot : executorToSlot
-                        .entrySet()) {
-                    WorkerSlot slot = execToSlot.getValue();
-                    ExecutorDetails exec = execToSlot.getKey();
-                    RAS_Node node = nodeIdToNode.get(slot.getNodeId());
-                    if (!node.isAlive()) {
-                        continue;
-                        // We do not free the assigned slots (the orphaned slots) on the inactive supervisors
-                        // The inactive node will be treated as a 0-resource node and not available for other unassigned workers
-                    }
-                    if (topoMemoryResourceList.containsKey(exec)) {
-                        node.consumeResourcesforTask(exec, topologies.getById(entry.getKey()));
-                    } else {
-                        LOG.warn("Resource Req not found...Scheduling Task{} with memory requirement as on heap - {} and off heap - {} and CPU requirement as {}",
-                                exec,
-                                Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB,
-                                Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT);
-                        topologies.getById(entry.getKey()).addDefaultResforExec(exec);
-                        node.consumeResourcesforTask(exec, topologies.getById(entry.getKey()));
-                    }
-                }
-            }
-        } else {
-            for (RAS_Node n : nodeIdToNode.values()) {
-                n.setAvailableMemory(n.getAvailableMemoryResources());
-            }
-        }
-    }
-
-    /**
      * Sets the Available Memory for a node
      * @param amount the amount to set as available memory
      */

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
new file mode 100644
index 0000000..9df1475
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
@@ -0,0 +1,149 @@
+/**
+ * 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 backtype.storm.scheduler.resource;
+
+import backtype.storm.Config;
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.ExecutorDetails;
+import backtype.storm.scheduler.SchedulerAssignment;
+import backtype.storm.scheduler.SupervisorDetails;
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.WorkerSlot;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+public class RAS_Nodes {
+
+    private Map<String, RAS_Node> nodeMap;
+    private Cluster cluster;
+    private Topologies topologies;
+
+    private static final Logger LOG = LoggerFactory.getLogger(RAS_Nodes.class);
+
+
+    public RAS_Nodes(Cluster cluster, Topologies topologies) {
+        this.nodeMap = getAllNodesFrom(cluster, topologies);
+        this.cluster = cluster;
+        this.topologies = topologies;
+    }
+
+    public static Map<String, RAS_Node> getAllNodesFrom(Cluster cluster, Topologies topologies) {
+        Map<String, RAS_Node> nodeIdToNode = new HashMap<String, RAS_Node>();
+        for (SupervisorDetails sup : cluster.getSupervisors().values()) {
+            //Node ID and supervisor ID are the same.
+            String id = sup.getId();
+            boolean isAlive = !cluster.isBlackListed(id);
+            LOG.debug("Found a {} Node {} {}",
+                    isAlive ? "living" : "dead", id, sup.getAllPorts());
+            LOG.debug("resources_mem: {}, resources_CPU: {}", sup.getTotalMemory(), sup.getTotalCPU());
+            nodeIdToNode.put(sup.getId(), new RAS_Node(id, sup.getAllPorts(), isAlive, sup, cluster));
+        }
+        for (Map.Entry<String, SchedulerAssignment> entry : cluster.getAssignments().entrySet()) {
+            String topId = entry.getValue().getTopologyId();
+            for (WorkerSlot workerSlot : entry.getValue().getSlots()) {
+                String id = workerSlot.getNodeId();
+                RAS_Node node = nodeIdToNode.get(id);
+                if (node == null) {
+                    LOG.info("Found an assigned slot on a dead supervisor {} with executors {}",
+                            workerSlot, RAS_Node.getExecutors(workerSlot, cluster));
+                    node = new RAS_Node(id, null, false, null, cluster);
+                    nodeIdToNode.put(id, node);
+                }
+                if (!node.isAlive()) {
+                    //The supervisor on the node down so add an orphaned slot to hold the unsupervised worker
+                    node.addOrphanedSlot(workerSlot);
+                }
+                if (node.assignInternal(workerSlot, topId, true)) {
+                    LOG.warn("Bad scheduling state, " + workerSlot + " assigned multiple workers, unassigning everything...");
+                    node.free(workerSlot);
+                }
+            }
+        }
+        updateAvailableResources(cluster, topologies, nodeIdToNode);
+        return nodeIdToNode;
+    }
+
+    /**
+     * updates the available resources for every node in a cluster
+     * by recalculating memory requirements.
+     * @param cluster the cluster used in this calculation
+     * @param topologies container of all topologies
+     * @param nodeIdToNode a map between node id and node
+     */
+    private static void updateAvailableResources(Cluster cluster,
+                                                 Topologies topologies,
+                                                 Map<String, RAS_Node> nodeIdToNode) {
+        //recompute memory
+        if (cluster.getAssignments().size() > 0) {
+            for (Map.Entry<String, SchedulerAssignment> entry : cluster.getAssignments()
+                    .entrySet()) {
+                Map<ExecutorDetails, WorkerSlot> executorToSlot = entry.getValue()
+                        .getExecutorToSlot();
+                Map<ExecutorDetails, Double> topoMemoryResourceList = topologies.getById(entry.getKey()).getTotalMemoryResourceList();
+
+                if (topoMemoryResourceList == null || topoMemoryResourceList.size() == 0) {
+                    continue;
+                }
+                for (Map.Entry<ExecutorDetails, WorkerSlot> execToSlot : executorToSlot
+                        .entrySet()) {
+                    WorkerSlot slot = execToSlot.getValue();
+                    ExecutorDetails exec = execToSlot.getKey();
+                    RAS_Node node = nodeIdToNode.get(slot.getNodeId());
+                    if (!node.isAlive()) {
+                        continue;
+                        // We do not free the assigned slots (the orphaned slots) on the inactive supervisors
+                        // The inactive node will be treated as a 0-resource node and not available for other unassigned workers
+                    }
+                    if (topoMemoryResourceList.containsKey(exec)) {
+                        node.consumeResourcesforTask(exec, topologies.getById(entry.getKey()));
+                    } else {
+                        LOG.warn("Resource Req not found...Scheduling Task{} with memory requirement as on heap - {} and off heap - {} and CPU requirement as {}",
+                                exec,
+                                Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB,
+                                Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT);
+                        topologies.getById(entry.getKey()).addDefaultResforExec(exec);
+                        node.consumeResourcesforTask(exec, topologies.getById(entry.getKey()));
+                    }
+                }
+            }
+        } else {
+            for (RAS_Node n : nodeIdToNode.values()) {
+                n.setAvailableMemory(n.getAvailableMemoryResources());
+            }
+        }
+    }
+
+    public RAS_Node getNodeById(String nodeId) {
+        return this.nodeMap.get(nodeId);
+    }
+
+    public void freeSlots(Collection<WorkerSlot> workerSlots) {
+        for(RAS_Node node : nodeMap.values()) {
+            for(WorkerSlot ws : node.getUsedSlots()) {
+                if(workerSlots.contains(ws)) {
+                    node.free(ws);
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
index 65d1841..279d060 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
@@ -19,6 +19,7 @@
 package backtype.storm.scheduler.resource;
 
 import backtype.storm.Config;
+import backtype.storm.scheduler.SchedulerAssignment;
 import backtype.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -39,9 +40,10 @@ import java.util.Set;
 
 public class ResourceAwareScheduler implements IScheduler {
 
-    private Map<String, User> userMap = new HashMap<String, User>();
+    private Map<String, User> userMap;
     private Cluster cluster;
     private Topologies topologies;
+    private RAS_Nodes nodes;
 
 
     @SuppressWarnings("rawtypes")
@@ -53,21 +55,26 @@ public class ResourceAwareScheduler implements IScheduler {
     @Override
     public void prepare(Map conf) {
         this.conf = conf;
+
     }
 
     @Override
     public void schedule(Topologies topologies, Cluster cluster) {
         LOG.info("\n\n\nRerunning ResourceAwareScheduler...");
         LOG.debug(ResourceUtils.printScheduling(cluster, topologies));
+        LOG.info("topologies: {}", topologies);
 
         this.initialize(topologies, cluster);
 
-
         LOG.info("UserMap:\n{}", this.userMap);
         for(User user : this.getUserMap().values()) {
             LOG.info(user.getDetailedInfo());
         }
 
+        for(TopologyDetails topo : topologies.getTopologies()) {
+            LOG.info("topo {} status: {}", topo, cluster.getStatusMap().get(topo.getId()));
+        }
+
         LOG.info("getNextUser: {}", this.getNextUser());
 
         while(true) {
@@ -80,15 +87,135 @@ public class ResourceAwareScheduler implements IScheduler {
         }
     }
 
-    private void scheduleTopology(TopologyDetails td) {
+    private boolean makeSpaceForTopo(TopologyDetails td) {
+        User submitter = this.userMap.get(td.getTopologySubmitter());
+        if (submitter.getCPUResourceGuaranteed() == null || submitter.getMemoryResourceGuaranteed() == null) {
+            return false;
+        }
+
+        double cpuNeeded = td.getTotalRequestedCpu() / submitter.getCPUResourceGuaranteed();
+        double memoryNeeded = (td.getTotalRequestedMemOffHeap() + td.getTotalRequestedMemOnHeap()) / submitter.getMemoryResourceGuaranteed();
+
+        //user has enough resource under his or her resource guarantee to schedule topology
+        if ((1.0 - submitter.getCPUResourcePoolUtilization()) > cpuNeeded && (1.0 - submitter.getMemoryResourcePoolUtilization()) > memoryNeeded) {
+            User evictUser = this.findUserWithMostResourcesAboveGuarantee();
+            if (evictUser == null) {
+                LOG.info("Cannot make space for topology {} from user {}", td.getName(), submitter.getId());
+                submitter.moveTopoFromPendingToAttempted(td, this.cluster);
+
+                return false;
+            }
+            TopologyDetails topologyEvict = evictUser.getRunningTopologyWithLowestPriority();
+            LOG.info("topology to evict: {}", topologyEvict);
+            evictTopology(topologyEvict);
+
+            return true;
+        } else {
+
+            if ((1.0 - submitter.getCPUResourcePoolUtilization()) < cpuNeeded) {
+
+            }
+
+            if ((1.0 - submitter.getMemoryResourcePoolUtilization()) < memoryNeeded) {
+
+            }
+            return false;
+
+        }
+    }
+
+    private void evictTopology(TopologyDetails topologyEvict) {
+        Collection<WorkerSlot> workersToEvict = this.cluster.getUsedSlotsByTopologyId(topologyEvict.getId());
+        User submitter = this.userMap.get(topologyEvict.getTopologySubmitter());
+
+        LOG.info("Evicting Topology {} with workers: {}", topologyEvict.getName(), workersToEvict);
+        this.nodes.freeSlots(workersToEvict);
+        submitter.moveTopoFromRunningToPending(topologyEvict, this.cluster);
+        LOG.info("check if topology unassigned: {}", this.cluster.getUsedSlotsByTopologyId(topologyEvict.getId()));
+    }
+
+    private User findUserWithMostResourcesAboveGuarantee() {
+        double most = 0.0;
+        User mostOverUser = null;
+        for(User user : this.userMap.values()) {
+            double over = user.getResourcePoolAverageUtilization() -1.0;
+            if((over > most) && (!user.getTopologiesRunning().isEmpty())) {
+                most = over;
+                mostOverUser = user;
+            }
+        }
+        return mostOverUser;
+    }
+
+    public void resetAssignments(Map<String, SchedulerAssignment> assignmentCheckpoint) {
+        this.cluster.setAssignments(assignmentCheckpoint);
+    }
+
+    public void scheduleTopology(TopologyDetails td) {
         ResourceAwareStrategy RAStrategy = new ResourceAwareStrategy(this.cluster, this.topologies);
-        if (cluster.needsScheduling(td) && cluster.getUnassignedExecutors(td).size() > 0) {
-            LOG.info("/********Scheduling topology {} from User {}************/", td.getName(), td.getTopologySubmitter());
+        User topologySubmitter = this.userMap.get(td.getTopologySubmitter());
+        if (cluster.getUnassignedExecutors(td).size() > 0) {
+            LOG.info("/********Scheduling topology {} from User {}************/", td.getName(), topologySubmitter);
             LOG.info("{}", this.userMap.get(td.getTopologySubmitter()).getDetailedInfo());
             LOG.info("{}", User.getResourcePoolAverageUtilizationForUsers(this.userMap.values()));
 
-            Map<WorkerSlot, Collection<ExecutorDetails>> schedulerAssignmentMap = RAStrategy.schedule(td);
+            Map<String, SchedulerAssignment> assignmentCheckpoint = this.cluster.getAssignments();
+
+            while (true) {
+                SchedulingResult result = RAStrategy.schedule(td);
+                LOG.info("scheduling result: {}", result);
+                if (result.isValid()) {
+                    if (result.isSuccess()) {
+                        try {
+                            if(mkAssignment(td, result.getSchedulingResultMap())) {
+                                topologySubmitter.moveTopoFromPendingToRunning(td, this.cluster);
+                            } else {
+                                resetAssignments(assignmentCheckpoint);
+                                topologySubmitter.moveTopoFromPendingToAttempted(td, this.cluster);
+                            }
+                        } catch (IllegalStateException ex) {
+                            LOG.error(ex.toString());
+                            LOG.error("Unsuccessful in scheduling", td.getId());
+                            this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
+                            resetAssignments(assignmentCheckpoint);
+                            topologySubmitter.moveTopoFromPendingToAttempted(td, this.cluster);
+                        }
+                        break;
+                    } else {
+                        if (result.getStatus() == SchedulingStatus.FAIL_NOT_ENOUGH_RESOURCES) {
+                            if(!this.makeSpaceForTopo(td)) {
+                                topologySubmitter.moveTopoFromPendingToAttempted(td);
+                                this.cluster.setStatus(td.getId(), result.getErrorMessage());
+                                resetAssignments(assignmentCheckpoint);
+                                break;
+                            }
+                            continue;
+                        } else if (result.getStatus() == SchedulingStatus.FAIL_INVALID_TOPOLOGY) {
+                            topologySubmitter.moveTopoFromPendingToInvalid(td, this.cluster);
+                            resetAssignments(assignmentCheckpoint);
+                            break;
+                        } else {
+                            topologySubmitter.moveTopoFromPendingToAttempted(td, this.cluster);
+                            resetAssignments(assignmentCheckpoint);
+                            break;
+                        }
+                    }
+                } else {
+                    LOG.warn("Scheduling results returned from topology {} is not vaild! Topology with be ignored.", td.getName());
+                    topologySubmitter.moveTopoFromPendingToInvalid(td, this.cluster);
+                    resetAssignments(assignmentCheckpoint);
+                    break;
+                }
+            }
+        } else {
+            LOG.warn("Topology {} is already fully scheduled!", td.getName());
+            topologySubmitter.moveTopoFromPendingToRunning(td, this.cluster);
+            throw new IllegalStateException("illegal");
+        }
+    }
 
+    private boolean mkAssignment(TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> schedulerAssignmentMap) {
+        if (schedulerAssignmentMap != null) {
             double requestedMemOnHeap = td.getTotalRequestedMemOnHeap();
             double requestedMemOffHeap = td.getTotalRequestedMemOffHeap();
             double requestedCpu = td.getTotalRequestedCpu();
@@ -96,39 +223,25 @@ public class ResourceAwareScheduler implements IScheduler {
             double assignedMemOffHeap = 0.0;
             double assignedCpu = 0.0;
 
-            if (schedulerAssignmentMap != null) {
-                try {
-                    Set<String> nodesUsed = new HashSet<String>();
-                    int assignedWorkers = schedulerAssignmentMap.keySet().size();
-                    for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> workerToTasksEntry : schedulerAssignmentMap.entrySet()) {
-                        WorkerSlot targetSlot = workerToTasksEntry.getKey();
-                        Collection<ExecutorDetails> execsNeedScheduling = workerToTasksEntry.getValue();
-                        RAS_Node targetNode = RAStrategy.idToNode(targetSlot.getNodeId());
-                        targetNode.assign(targetSlot, td, execsNeedScheduling, this.cluster);
-                        LOG.debug("ASSIGNMENT    TOPOLOGY: {}  TASKS: {} To Node: {} on Slot: {}",
-                                td.getName(), execsNeedScheduling, targetNode.getHostname(), targetSlot.getPort());
-                        if (!nodesUsed.contains(targetNode.getId())) {
-                            nodesUsed.add(targetNode.getId());
-                        }
-                        assignedMemOnHeap += targetSlot.getAllocatedMemOnHeap();
-                        assignedMemOffHeap += targetSlot.getAllocatedMemOffHeap();
-                        assignedCpu += targetSlot.getAllocatedCpu();
-                    }
-                    LOG.debug("Topology: {} assigned to {} nodes on {} workers", td.getId(), nodesUsed.size(), assignedWorkers);
-                    this.cluster.setStatus(td.getId(), "Fully Scheduled");
-                    this.getUser(td.getTopologySubmitter()).moveTopoFromPendingToRunning(td);
-                    LOG.info("getNextUser: {}", this.getNextUser());
-                } catch (IllegalStateException ex) {
-                    LOG.error(ex.toString());
-                    LOG.error("Unsuccessful in scheduling", td.getId());
-                    this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
-                    this.getUser(td.getTopologySubmitter()).moveTopoFromPendingToAttempted(td);
+            Set<String> nodesUsed = new HashSet<String>();
+            int assignedWorkers = schedulerAssignmentMap.keySet().size();
+            for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> workerToTasksEntry : schedulerAssignmentMap.entrySet()) {
+                WorkerSlot targetSlot = workerToTasksEntry.getKey();
+                Collection<ExecutorDetails> execsNeedScheduling = workerToTasksEntry.getValue();
+                RAS_Node targetNode = this.nodes.getNodeById(targetSlot.getNodeId());
+                targetNode.assign(targetSlot, td, execsNeedScheduling);
+                LOG.debug("ASSIGNMENT    TOPOLOGY: {}  TASKS: {} To Node: {} on Slot: {}",
+                        td.getName(), execsNeedScheduling, targetNode.getHostname(), targetSlot.getPort());
+                if (!nodesUsed.contains(targetNode.getId())) {
+                    nodesUsed.add(targetNode.getId());
                 }
-            } else {
-                LOG.error("Unsuccessful in scheduling {}", td.getId());
-                this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
-                this.getUser(td.getTopologySubmitter()).moveTopoFromPendingToAttempted(td);
+                assignedMemOnHeap += targetSlot.getAllocatedMemOnHeap();
+                assignedMemOffHeap += targetSlot.getAllocatedMemOffHeap();
+                assignedCpu += targetSlot.getAllocatedCpu();
             }
+            LOG.debug("Topology: {} assigned to {} nodes on {} workers", td.getId(), nodesUsed.size(), assignedWorkers);
+            this.cluster.setStatus(td.getId(), "Fully Scheduled");
+
             Double[] resources = {requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
                     assignedMemOnHeap, assignedMemOffHeap, assignedCpu};
             LOG.debug("setResources for {}: requested on-heap mem, off-heap mem, cpu: {} {} {} " +
@@ -136,9 +249,10 @@ public class ResourceAwareScheduler implements IScheduler {
                     td.getId(), requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
                     assignedMemOnHeap, assignedMemOffHeap, assignedCpu);
             this.cluster.setResources(td.getId(), resources);
+            return true;
         } else {
-            LOG.warn("Topology {} already scheduled!", td.getName());
-            this.cluster.setStatus(td.getId(), "Fully Scheduled");
+            LOG.warn("schedulerAssignmentMap for topo {} is null. This shouldn't happen!", td.getName());
+            return false;
         }
         updateSupervisorsResources(cluster, topologies);
     }
@@ -157,6 +271,70 @@ public class ResourceAwareScheduler implements IScheduler {
         }
         cluster.setSupervisorsResourcesMap(supervisors_resources);
     }
+
+
+//    private void scheduleTopology(TopologyDetails td) {
+//        ResourceAwareStrategy RAStrategy = new ResourceAwareStrategy(this.cluster, this.topologies);
+//        if (cluster.needsScheduling(td) && cluster.getUnassignedExecutors(td).size() > 0) {
+//            LOG.info("/********Scheduling topology {} from User {}************/", td.getName(), td.getTopologySubmitter());
+//            LOG.info("{}", this.userMap.get(td.getTopologySubmitter()).getDetailedInfo());
+//            LOG.info("{}", User.getResourcePoolAverageUtilizationForUsers(this.userMap.values()));
+//
+//            Map<WorkerSlot, Collection<ExecutorDetails>> schedulerAssignmentMap = RAStrategy.schedule(td);
+//
+//            double requestedMemOnHeap = td.getTotalRequestedMemOnHeap();
+//            double requestedMemOffHeap = td.getTotalRequestedMemOffHeap();
+//            double requestedCpu = td.getTotalRequestedCpu();
+//            double assignedMemOnHeap = 0.0;
+//            double assignedMemOffHeap = 0.0;
+//            double assignedCpu = 0.0;
+//
+//            if (schedulerAssignmentMap != null) {
+//                try {
+//                    Set<String> nodesUsed = new HashSet<String>();
+//                    int assignedWorkers = schedulerAssignmentMap.keySet().size();
+//                    for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> workerToTasksEntry : schedulerAssignmentMap.entrySet()) {
+//                        WorkerSlot targetSlot = workerToTasksEntry.getKey();
+//                        Collection<ExecutorDetails> execsNeedScheduling = workerToTasksEntry.getValue();
+//                        RAS_Node targetNode = RAStrategy.idToNode(targetSlot.getNodeId());
+//                        targetNode.assign(targetSlot, td, execsNeedScheduling, this.cluster);
+//                        LOG.debug("ASSIGNMENT    TOPOLOGY: {}  TASKS: {} To Node: {} on Slot: {}",
+//                                td.getName(), execsNeedScheduling, targetNode.getHostname(), targetSlot.getPort());
+//                        if (!nodesUsed.contains(targetNode.getId())) {
+//                            nodesUsed.add(targetNode.getId());
+//                        }
+//                        assignedMemOnHeap += targetSlot.getAllocatedMemOnHeap();
+//                        assignedMemOffHeap += targetSlot.getAllocatedMemOffHeap();
+//                        assignedCpu += targetSlot.getAllocatedCpu();
+//                    }
+//                    LOG.debug("Topology: {} assigned to {} nodes on {} workers", td.getId(), nodesUsed.size(), assignedWorkers);
+//                    this.cluster.setStatus(td.getId(), "Fully Scheduled");
+//                    this.getUser(td.getTopologySubmitter()).moveTopoFromPendingToRunning(td);
+//                    LOG.info("getNextUser: {}", this.getNextUser());
+//                } catch (IllegalStateException ex) {
+//                    LOG.error(ex.toString());
+//                    LOG.error("Unsuccessful in scheduling", td.getId());
+//                    this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
+//                    this.getUser(td.getTopologySubmitter()).moveTopoFromPendingToAttempted(td);
+//                }
+//            } else {
+//                LOG.error("Unsuccessful in scheduling {}", td.getId());
+//                this.cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
+//              //  this.evictTopology(td);
+//               // this.getUser(td.getTopologySubmitter()).moveTopoFromPendingToAttempted(td);
+//            }
+//            Double[] resources = {requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
+//                    assignedMemOnHeap, assignedMemOffHeap, assignedCpu};
+//            LOG.debug("setResources for {}: requested on-heap mem, off-heap mem, cpu: {} {} {} " +
+//                            "assigned on-heap mem, off-heap mem, cpu: {} {} {}",
+//                    td.getId(), requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
+//                    assignedMemOnHeap, assignedMemOffHeap, assignedCpu);
+//            this.cluster.setResources(td.getId(), resources);
+//        } else {
+//            LOG.warn("Topology {} already scheduled!", td.getName());
+//            this.cluster.setStatus(td.getId(), "Fully Scheduled");
+//        }
+//    }
     public User getUser(String user) {
         return this.userMap.get(user);
     }
@@ -166,9 +344,11 @@ public class ResourceAwareScheduler implements IScheduler {
     }
 
     public User getNextUser() {
-        Double least = Double.MAX_VALUE;
+        Double least = Double.POSITIVE_INFINITY;
         User ret = null;
         for(User user : this.userMap.values()) {
+            LOG.info("{}", user.getDetailedInfo());
+            LOG.info("hasTopologyNeedSchedule: {}", user.hasTopologyNeedSchedule());
             if(user.hasTopologyNeedSchedule()) {
                 Double userResourcePoolAverageUtilization = user.getResourcePoolAverageUtilization();
                 if (least > userResourcePoolAverageUtilization) {
@@ -199,18 +379,25 @@ public class ResourceAwareScheduler implements IScheduler {
      */
     private void initUsers(Topologies topologies, Cluster cluster) {
 
+        this.userMap = new HashMap<String, User>();
         Map<String, Map<String, Double>> userResourcePools = this.getUserResourcePools();
         LOG.info("userResourcePools: {}", userResourcePools);
 
-        for (TopologyDetails topo : topologies.getTopologies()) {
-            String topologySubmitter = topo.getTopologySubmitter();
+        for (TopologyDetails td : topologies.getTopologies()) {
+            LOG.info("topology: {} from {}", td.getName(), td.getTopologySubmitter());
+            String topologySubmitter = td.getTopologySubmitter();
+            if(topologySubmitter == null) {
+                LOG.warn("Topology {} submitted by anonymous user", td.getName());
+                topologySubmitter = "anonymous";
+            }
             if(!this.userMap.containsKey(topologySubmitter)) {
                 this.userMap.put(topologySubmitter, new User(topologySubmitter, userResourcePools.get(topologySubmitter)));
             }
-            if(cluster.getUnassignedExecutors(topo).size() >= topo.getExecutors().size()) {
-                this.userMap.get(topologySubmitter).addTopologyToPendingQueue(topo);
+            if(cluster.getUnassignedExecutors(td).size() >= td.getExecutors().size()) {
+                this.userMap.get(topologySubmitter).addTopologyToPendingQueue(td, cluster);
+                LOG.info(this.userMap.get(topologySubmitter).getDetailedInfo());
             } else {
-                this.userMap.get(topologySubmitter).addTopologyToRunningQueue(topo);
+                this.userMap.get(topologySubmitter).addTopologyToRunningQueue(td, cluster);
             }
         }
     }
@@ -219,6 +406,7 @@ public class ResourceAwareScheduler implements IScheduler {
         initUsers(topologies, cluster);
         this.cluster = cluster;
         this.topologies = topologies;
+        this.nodes = new RAS_Nodes(this.cluster, this.topologies);
     }
 
     /**
@@ -226,14 +414,13 @@ public class ResourceAwareScheduler implements IScheduler {
      * @return
      */
     private Map<String, Map<String, Double>> getUserResourcePools() {
-
-        Map<String, Map<String, Number>> raw =  (Map<String, Map<String, Number>>)this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS);
+        Object raw = this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS);
         Map<String, Map<String, Double>> ret =  (Map<String, Map<String, Double>>)this.conf.get(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS);
 
         if (raw == null) {
-            ret = new  HashMap<String, Map<String, Double>>();
+            ret = new HashMap<String, Map<String, Double>>();
         } else {
-            for(Map.Entry<String, Map<String, Number>> UserPoolEntry : raw.entrySet()) {
+            for(Map.Entry<String, Map<String, Number>> UserPoolEntry : ((Map<String, Map<String, Number>>) raw).entrySet()) {
                 String user = UserPoolEntry.getKey();
                 ret.put(user, new HashMap<String, Double>());
                 for(Map.Entry<String, Number> resourceEntry : UserPoolEntry.getValue().entrySet()) {

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingResult.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingResult.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingResult.java
new file mode 100644
index 0000000..9e7b1ff
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingResult.java
@@ -0,0 +1,111 @@
+/**
+ * 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 backtype.storm.scheduler.resource;
+
+import backtype.storm.scheduler.ExecutorDetails;
+import backtype.storm.scheduler.WorkerSlot;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Map;
+
+public class SchedulingResult {
+
+    //contains the result for the attempted scheduling
+    private Map<WorkerSlot, Collection<ExecutorDetails>> schedulingResultMap = null;
+
+    private SchedulingStatus status = null;
+
+    private String message = null;
+
+    private String errorMessage = null;
+
+    private static final Logger LOG = LoggerFactory.getLogger(SchedulingResult.class);
+
+
+    public SchedulingResult(SchedulingStatus status, Map<WorkerSlot, Collection<ExecutorDetails>> schedulingResultMap, String message, String errorMessage) {
+        this.status = status;
+        this.schedulingResultMap = schedulingResultMap;
+        this.message = message;
+        this.errorMessage = errorMessage;
+    }
+
+    public static SchedulingResult failure(SchedulingStatus status, String errorMessage) {
+        return new SchedulingResult(status, null, null, errorMessage);
+    }
+
+    public static SchedulingResult success(Map<WorkerSlot, Collection<ExecutorDetails>> schedulingResultMap) {
+        return SchedulingResult.successWithMsg(schedulingResultMap, null);
+    }
+
+    public static SchedulingResult successWithMsg(Map<WorkerSlot, Collection<ExecutorDetails>> schedulingResultMap, String message) {
+        if (schedulingResultMap == null) {
+            throw new IllegalStateException("Cannot declare scheduling success without providing a non null scheduling map!");
+        }
+        return new SchedulingResult(SchedulingStatus.SUCCESS, schedulingResultMap, message, null);
+    }
+
+    public SchedulingStatus getStatus() {
+        return this.status;
+    }
+
+    public String getMessage() {
+        return this.message;
+    }
+
+    public String getErrorMessage() {
+        return this.errorMessage;
+    }
+
+    public Map<WorkerSlot, Collection<ExecutorDetails>> getSchedulingResultMap() {
+        return schedulingResultMap;
+    }
+
+    public boolean isSuccess() {
+        return SchedulingStatus.isStatusSuccess(this.status);
+    }
+
+    public boolean isFailure() {
+        return SchedulingStatus.isStatusFailure(this.status);
+    }
+
+    public boolean isValid() {
+        if (this.isSuccess() && this.getSchedulingResultMap() == null) {
+            LOG.warn("SchedulingResult not Valid! Status is success but SchedulingResultMap is null");
+            return false;
+        }
+        if (this.isFailure() && this.getSchedulingResultMap() != null) {
+            LOG.warn("SchedulingResult not Valid! Status is Failure but SchedulingResultMap is NOT null");
+            return false;
+        }
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        String ret = "";
+        if(this.isSuccess()) {
+            ret += "Status: " + this.getStatus() + " message: " + this.getMessage() + " scheduling: " + this.getSchedulingResultMap().toString();
+        } else {
+            ret += "Status: " + this.getStatus() + " error message: " + this.getErrorMessage();
+        }
+        return ret;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingStatus.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingStatus.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingStatus.java
new file mode 100644
index 0000000..4622c29
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/SchedulingStatus.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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 backtype.storm.scheduler.resource;
+
+import java.util.EnumSet;
+
+public enum SchedulingStatus {
+    SUCCESS,
+    FAIL_NOT_ENOUGH_RESOURCES,
+    FAIL_INVALID_TOPOLOGY,
+    FAIL_OTHER;
+
+    public static EnumSet<SchedulingStatus> success = EnumSet.of(SUCCESS);
+    public static EnumSet<SchedulingStatus> failure = EnumSet.of(FAIL_INVALID_TOPOLOGY, FAIL_NOT_ENOUGH_RESOURCES, FAIL_OTHER);
+
+    public static boolean isStatusSuccess(SchedulingStatus status) {
+        return success.contains(status);
+    }
+
+    public static boolean isStatusFailure(SchedulingStatus status) {
+        return failure.contains(status);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
index 32af686..068db54 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/User.java
@@ -18,6 +18,7 @@
 
 package backtype.storm.scheduler.resource;
 
+import backtype.storm.scheduler.Cluster;
 import backtype.storm.scheduler.TopologyDetails;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -42,6 +43,8 @@ public class User {
     //Topologies that was attempted to be scheduled but wasn't successull
     private Set<TopologyDetails> attemptedQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
 
+    private Set<TopologyDetails> invalidQueue = new TreeSet<TopologyDetails>(new PQsortByPriorityAndSubmittionTime());
+
     private Map<String, Double> resourcePool = new HashMap<String, Double>();
 
     private static final Logger LOG = LoggerFactory.getLogger(User.class);
@@ -61,12 +64,26 @@ public class User {
         return this.userId;
     }
 
-    public void addTopologyToPendingQueue(TopologyDetails topo) {
+    public void addTopologyToPendingQueue(TopologyDetails topo, Cluster cluster) {
         this.pendingQueue.add(topo);
+        if (cluster != null) {
+            cluster.setStatus(topo.getId(), "Scheduling Pending");
+        }
     }
 
-    public void addTopologyToRunningQueue(TopologyDetails topo) {
+    public void addTopologyToPendingQueue(TopologyDetails topo) {
+        this.addTopologyToPendingQueue(topo, null);
+    }
+
+    public void addTopologyToRunningQueue(TopologyDetails topo, Cluster cluster) {
         this.runningQueue.add(topo);
+        if (cluster != null) {
+            cluster.setStatus(topo.getId(), "Fully Scheduled");
+        }
+    }
+
+    public void addTopologyToRunningQueue(TopologyDetails topo) {
+        this.addTopologyToRunningQueue(topo, null);
     }
 
     public Set<TopologyDetails> getTopologiesPending() {
@@ -94,23 +111,69 @@ public class User {
         return null;
     }
 
-    public void moveTopoFromPendingToRunning(TopologyDetails topo) {
+    public void moveTopoFromPendingToRunning(TopologyDetails topo, Cluster cluster) {
         moveTopology(topo, this.pendingQueue, "pending", this.runningQueue, "running");
+        if (cluster != null) {
+            cluster.setStatus(topo.getId(), "Fully Scheduled");
+        }
     }
 
-    public void moveTopoFromPendingToAttempted(TopologyDetails topo) {
+    public void moveTopoFromPendingToRunning(TopologyDetails topo) {
+        this.moveTopoFromPendingToRunning(topo, null);
+    }
+
+
+    public void moveTopoFromPendingToAttempted(TopologyDetails topo, Cluster cluster) {
         moveTopology(topo, this.pendingQueue, "pending", this.attemptedQueue, "attempted");
+        if (cluster != null) {
+            cluster.setStatus(topo.getId(), "Scheduling Attempted but Failed");
+        }
+    }
+
+    public void moveTopoFromPendingToAttempted(TopologyDetails topo) {
+        this.moveTopoFromPendingToAttempted(topo, null);
     }
 
-    private void moveTopology(TopologyDetails topo, Set<TopologyDetails> src, String srcName, Set<TopologyDetails> dest, String destName)  {
-        if(topo == null) {
+
+    public void moveTopoFromPendingToInvalid(TopologyDetails topo, Cluster cluster) {
+        moveTopology(topo, this.pendingQueue, "pending", this.invalidQueue, "invalid");
+        if (cluster != null) {
+            cluster.setStatus(topo.getId(), "Scheduling Attempted but topology is invalid");
+        }
+    }
+
+    public void moveTopoFromPendingToInvalid(TopologyDetails topo) {
+        this.moveTopoFromPendingToInvalid(topo, null);
+    }
+
+
+    public void moveTopoFromRunningToPending(TopologyDetails topo, Cluster cluster) {
+        moveTopology(topo, this.runningQueue, "running", this.pendingQueue, "pending");
+        if (cluster != null) {
+            cluster.setStatus(topo.getId(), "Scheduling Pending");
+        }
+    }
+
+    public void moveTopoFromRunningToPending(TopologyDetails topo) {
+        this.moveTopoFromRunningToPending(topo, null);
+    }
+
+
+    private void moveTopology(TopologyDetails topo, Set<TopologyDetails> src, String srcName, Set<TopologyDetails> dest, String destName) {
+        LOG.info("{} queue: {}", srcName, src);
+        LOG.info("{} queue: {}", destName, dest);
+        if (topo == null) {
             return;
         }
-        if(!src.contains(topo)) {
+        if (!src.contains(topo)) {
             LOG.warn("Topo {} not in User: {} {} queue!", topo.getName(), this.userId, srcName);
+            LOG.info("topo {}-{}-{}", topo.getName(), topo.getId(), topo.hashCode());
+            for (TopologyDetails t : src) {
+                LOG.info("queue entry: {}-{}-{}", t.getName(), t.getId(), t.hashCode());
+            }
             return;
         }
-        if(dest.contains(topo)) {
+        if (dest.contains(topo)) {
             LOG.warn("Topo {} already in in User: {} {} queue!", topo.getName(), this.userId, destName);
             return;
         }
@@ -124,8 +187,8 @@ public class User {
         Double cpuResourcePoolUtilization = this.getCPUResourcePoolUtilization();
         Double memoryResourcePoolUtilization = this.getMemoryResourcePoolUtilization();
 
-        if(cpuResourcePoolUtilization != null && memoryResourcePoolUtilization != null) {
-            return (cpuResourcePoolUtilization + memoryResourcePoolUtilization ) / 2.0;
+        if (cpuResourcePoolUtilization != null && memoryResourcePoolUtilization != null) {
+            return (cpuResourcePoolUtilization + memoryResourcePoolUtilization) / 2.0;
         }
         return Double.MAX_VALUE;
     }
@@ -173,7 +236,7 @@ public class User {
 
     public TopologyDetails getNextTopologyToSchedule() {
         for (TopologyDetails topo : this.pendingQueue) {
-            if(!this.attemptedQueue.contains(topo)) {
+            if (!this.attemptedQueue.contains(topo)) {
                 return topo;
             }
         }
@@ -184,6 +247,13 @@ public class User {
         return (!this.pendingQueue.isEmpty() && (this.pendingQueue.size() - this.attemptedQueue.size()) > 0);
     }
 
+    public TopologyDetails getRunningTopologyWithLowestPriority() {
+        if (this.runningQueue.isEmpty()) {
+            return null;
+        }
+        return this.runningQueue.iterator().next();
+    }
+
     @Override
     public int hashCode() {
         return this.userId.hashCode();
@@ -197,9 +267,10 @@ public class User {
     public String getDetailedInfo() {
         String ret = "\nUser: " + this.userId;
         ret += "\n - " + " Resource Pool: " + this.resourcePool;
-        ret += "\n - " + " Running Queue: " + this.runningQueue;
-        ret += "\n - " + " Pending Queue: " + this.pendingQueue;
-        ret += "\n - " + " Attempted Queue: " + this.attemptedQueue;
+        ret += "\n - " + " Running Queue: " + this.runningQueue + " size: " + this.runningQueue.size();
+        ret += "\n - " + " Pending Queue: " + this.pendingQueue + " size: " + this.pendingQueue.size();
+        ret += "\n - " + " Attempted Queue: " + this.attemptedQueue + " size: " + this.attemptedQueue.size();
+        ret += "\n - " + " Invalid Queue: " + this.invalidQueue + " size: " + this.invalidQueue.size();
         ret += "\n - " + " CPU Used: " + this.getCPUResourceUsedByUser() + " CPU guaranteed: " + this.getCPUResourceGuaranteed();
         ret += "\n - " + " Memory Used: " + this.getMemoryResourceUsedByUser() + " Memory guaranteed: " + this.getMemoryResourceGuaranteed();
         ret += "\n - " + " % Resource Guarantee Used: \n -- CPU: " + this.getCPUResourcePoolUtilization()
@@ -209,12 +280,31 @@ public class User {
 
     public static String getResourcePoolAverageUtilizationForUsers(Collection<User> users) {
         String ret = "";
-        for(User user : users) {
+        for (User user : users) {
             ret += user.getId() + " - " + user.getResourcePoolAverageUtilization() + " ";
         }
         return ret;
     }
 
+    public static int cTo(TopologyDetails topo1, TopologyDetails topo2) {
+        if (topo1.getId().compareTo(topo2.getId()) == 0) {
+            return 0;
+        }
+        if (topo1.getTopologyPriority() > topo2.getTopologyPriority()) {
+            return 1;
+        } else if (topo1.getTopologyPriority() < topo2.getTopologyPriority()) {
+            return -1;
+        } else {
+            if (topo1.getUpTime() > topo2.getUpTime()) {
+                return -1;
+            } else if (topo1.getUpTime() < topo2.getUpTime()) {
+                return 1;
+            } else {
+                return topo1.getId().compareTo(topo2.getId());
+            }
+        }
+    }
+
     /**
      * Comparator that sorts topologies by priority and then by submission time
      */
@@ -231,7 +321,7 @@ public class User {
                 } else if (topo1.getUpTime() < topo2.getUpTime()) {
                     return 1;
                 } else {
-                    return 0;
+                    return topo1.getId().compareTo(topo2.getId());
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/IStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/IStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/IStrategy.java
index 01f3223..722eddb 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/IStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/IStrategy.java
@@ -26,6 +26,7 @@ import backtype.storm.scheduler.ExecutorDetails;
 import backtype.storm.scheduler.TopologyDetails;
 import backtype.storm.scheduler.WorkerSlot;
 import backtype.storm.scheduler.resource.RAS_Node;
+import backtype.storm.scheduler.resource.SchedulingResult;
 
 /**
  * An interface to for implementing different scheduling strategies for the resource aware scheduling
@@ -33,5 +34,5 @@ import backtype.storm.scheduler.resource.RAS_Node;
  */
 public interface IStrategy {
 
-    public Map<WorkerSlot, Collection<ExecutorDetails>> schedule(TopologyDetails td);
+    public SchedulingResult schedule(TopologyDetails td);
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/3f55feef/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/ResourceAwareStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/ResourceAwareStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/ResourceAwareStrategy.java
index 3fe37dd..812bf5d 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/ResourceAwareStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/ResourceAwareStrategy.java
@@ -30,6 +30,9 @@ import java.util.TreeMap;
 import java.util.HashSet;
 import java.util.Iterator;
 
+import backtype.storm.scheduler.resource.RAS_Nodes;
+import backtype.storm.scheduler.resource.SchedulingResult;
+import backtype.storm.scheduler.resource.SchedulingStatus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -61,7 +64,7 @@ public class ResourceAwareStrategy implements IStrategy {
     public ResourceAwareStrategy(Cluster cluster, Topologies topologies) {
         _topologies = topologies;
         _cluster = cluster;
-        _nodes = RAS_Node.getAllNodesFrom(cluster, _topologies);
+        _nodes = RAS_Nodes.getAllNodesFrom(cluster, _topologies);
         _availNodes = this.getAvailNodes();
         _clusterInfo = cluster.getNetworkTopography();
         LOG.debug(this.getClusterInfo());
@@ -84,10 +87,10 @@ public class ResourceAwareStrategy implements IStrategy {
         return retMap;
     }
 
-    public Map<WorkerSlot, Collection<ExecutorDetails>> schedule(TopologyDetails td) {
+    public SchedulingResult schedule(TopologyDetails td) {
         if (_availNodes.size() <= 0) {
             LOG.warn("No available nodes to schedule tasks on!");
-            return null;
+            return SchedulingResult.failure(SchedulingStatus.FAIL_NOT_ENOUGH_RESOURCES, "No available nodes to schedule tasks on!");
         }
         Collection<ExecutorDetails> unassignedExecutors = _cluster.getUnassignedExecutors(td);
         Map<WorkerSlot, Collection<ExecutorDetails>> schedulerAssignmentMap = new HashMap<>();
@@ -97,7 +100,7 @@ public class ResourceAwareStrategy implements IStrategy {
 
         if (spouts.size() == 0) {
             LOG.error("Cannot find a Spout!");
-            return null;
+            return SchedulingResult.failure(SchedulingStatus.FAIL_INVALID_TOPOLOGY, "Cannot find a Spout!");
         }
 
         Queue<Component> ordered__Component_list = bfs(_topologies, td, spouts);
@@ -159,18 +162,22 @@ public class ResourceAwareStrategy implements IStrategy {
                 LOG.error("Not Enough Resources to schedule Task {}", exec);
             }
         }
+
+        SchedulingResult result;
         executorsNotScheduled.removeAll(scheduledTasks);
         if (executorsNotScheduled.size() > 0) {
             LOG.error("Not all executors successfully scheduled: {}",
                     executorsNotScheduled);
             schedulerAssignmentMap = null;
+            result = SchedulingResult.failure(SchedulingStatus.FAIL_NOT_ENOUGH_RESOURCES, "Not all executors successfully scheduled: " + executorsNotScheduled);
         } else {
             LOG.debug("All resources successfully scheduled!");
+            result = SchedulingResult.success(schedulerAssignmentMap);
         }
         if (schedulerAssignmentMap == null) {
             LOG.error("Topology {} not successfully scheduled!", td.getId());
         }
-        return schedulerAssignmentMap;
+        return result;
     }
 
     private WorkerSlot findWorkerForExec(ExecutorDetails exec, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {


[13/23] storm git commit: make fixes based on review comments

Posted by da...@apache.org.
make fixes based on review comments


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

Branch: refs/heads/master
Commit: e45ccfe765fdbe5ed48dba9952bbd82aa69061c4
Parents: f5bccd0
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Wed Dec 9 10:27:03 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Wed Dec 9 10:27:03 2015 -0600

----------------------------------------------------------------------
 conf/defaults.yaml                              |   2 +-
 .../Resource_Aware_Scheduler_overview.md        |  20 +++++-----
 ...ware_scheduler_default_eviction_strategy.png | Bin 188099 -> 0 bytes
 ...ware_scheduler_default_eviction_strategy.svg |   3 ++
 .../resource/TestResourceAwareScheduler.java    |  38 +++++++++----------
 5 files changed, 34 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/e45ccfe7/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index cef09d3..448db46 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -249,9 +249,9 @@ topology.disruptor.wait.timeout.millis: 1000
 topology.disruptor.batch.size: 100
 topology.disruptor.batch.timeout.millis: 1
 topology.disable.loadaware: false
-topology.priority: 30
 
 # Configs for Resource Aware Scheduler
+topology.priority: 29
 topology.component.resources.onheap.memory.mb: 128.0
 topology.component.resources.offheap.memory.mb: 0.0
 topology.component.cpu.pcore.percent: 10.0

http://git-wip-us.apache.org/repos/asf/storm/blob/e45ccfe7/docs/documentation/Resource_Aware_Scheduler_overview.md
----------------------------------------------------------------------
diff --git a/docs/documentation/Resource_Aware_Scheduler_overview.md b/docs/documentation/Resource_Aware_Scheduler_overview.md
index 0ca8d09..f0ffd44 100644
--- a/docs/documentation/Resource_Aware_Scheduler_overview.md
+++ b/docs/documentation/Resource_Aware_Scheduler_overview.md
@@ -21,9 +21,9 @@ API to set component memory requirement:
 
 Parameters:
 * Number onHeap – The amount of on heap memory an instance of this component will consume in megabytes
-* Number OffHeap – The amount of off heap memory an instance of this component will consume in megabytes
+* Number offHeap – The amount of off heap memory an instance of this component will consume in megabytes
 
-The user also have to option to just specify the on heap memory requirement if the component does not have an off heap memory need.
+The user also has to option to just specify the on heap memory requirement if the component does not have an off heap memory need.
 
     public T setMemoryLoad(Number onHeap)
 
@@ -98,7 +98,7 @@ Example of Usage:
     supervisor.cpu.capacity: 100.0
 
 
-2.5.	Other Configurations
+### Other Configurations
 
 The user can set some default configurations for the Resource Aware Scheduler in *conf/storm.yaml*:
 
@@ -143,8 +143,8 @@ An example of what *user-resource-pools.yaml* can look like:
 Please note that the specified amount of Guaranteed CPU and Memory can be either a integer or double
 
 ## API Overview
-### Specifying topology priority
-The range of topology priorities can range form 0-30.  The topologies priorities will be partitioned into several priority levels that may contain a range of priorities. 
+### Specifying Topology Priority
+The range of topology priorities can range form 0-29.  The topologies priorities will be partitioned into several priority levels that may contain a range of priorities. 
 For example we can create a priority level mapping:
 
     PRODUCTION => 0 – 9
@@ -154,6 +154,7 @@ For example we can create a priority level mapping:
 Thus, each priority level contains 10 sub priorities. Users can set the priority level of a topology by using the following API
 
     conf.setTopologyPriority(int priority)
+    
 Parameters:
 * priority – an integer representing the priority of the topology
 
@@ -167,6 +168,7 @@ Parameters:
 * clazz – The strategy class that implements the IStrategy interface
 
 Example Usage:
+
     conf.setTopologyStrategy(backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class);
 
 A default scheduling is provided.  The DefaultResourceAwareStrategy is implemented based off the scheduling algorithm in the original paper describing resource aware scheduling in Storm:
@@ -179,11 +181,11 @@ The order of scheduling is a pluggable interface in which a user could define a
 
     resource.aware.scheduler.priority.strategy: "backtype.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy"
     
-    A default strategy will be provided.  The following explains how the default scheduling priority strategy works.
+A default strategy will be provided.  The following explains how the default scheduling priority strategy works.
 
 **DefaultSchedulingPriorityStrategy**
 
-The order of scheduling should be based on how far is a user’s current resource allocation to his or her guaranteed allocation.  We should prioritize the users who are the furthest away from their resource guarantee. The difficulty of this problem is that a user may have multiple resource guarantees and another user can have another set resource guarantees, so how can we compare them in a fair manner?  Lets use the average percentage of resource guarantees satisfied as a method of comparison.
+The order of scheduling should be based on the distance a user’s current resource allocation to his or her guaranteed allocation.  We should prioritize the users who are the furthest away from their resource guarantee. The difficulty of this problem is that a user may have multiple resource guarantees, and another user can have another set resource guarantees, so how can we compare them in a fair manner?  Let's use the average percentage of resource guarantees satisfied as a method of comparison.
 
 For example:
 
@@ -213,10 +215,10 @@ A default eviction strategy is provided.  The following explains how the default
 
 **DefaultEvictionStrategy**
 
-If the cluster is full, we need a mechanism to evict topologies so that user resource guarantees can be met and resources additional resource guarantees can be shared fairly among users
+If the cluster is full, we need a mechanism to evict topologies so that user resource guarantees can be met and additional resource can be shared fairly among users
 
 To determine if topology eviction should occur we should take into account the priority of the topology that we are trying to schedule and whether the resource guarantees for the owner of the topology have been met.  
 
 We should never evict a topology from a user that does not have his or her resource guarantees satisfied.  The following flow chart should describe the logic for the eviction process.
 
-![Viewing metrics with VisualVM](images/resource_aware_scheduler_default_eviction_strategy.png)
\ No newline at end of file
+![Viewing metrics with VisualVM](images/resource_aware_scheduler_default_eviction_strategy.svg)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e45ccfe7/docs/images/resource_aware_scheduler_default_eviction_strategy.png
----------------------------------------------------------------------
diff --git a/docs/images/resource_aware_scheduler_default_eviction_strategy.png b/docs/images/resource_aware_scheduler_default_eviction_strategy.png
deleted file mode 100644
index 3c07e1f..0000000
Binary files a/docs/images/resource_aware_scheduler_default_eviction_strategy.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/storm/blob/e45ccfe7/docs/images/resource_aware_scheduler_default_eviction_strategy.svg
----------------------------------------------------------------------
diff --git a/docs/images/resource_aware_scheduler_default_eviction_strategy.svg b/docs/images/resource_aware_scheduler_default_eviction_strategy.svg
new file mode 100644
index 0000000..8a09c85
--- /dev/null
+++ b/docs/images/resource_aware_scheduler_default_eviction_strategy.svg
@@ -0,0 +1,3 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<svg xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" width="2238px" height="1523px" version="1.1" style="background-color: rgb(255, 255, 255);"><defs/><g transform="translate(0.5,0.5)"><path d="M 988.5 162 L 1042 212 L 988.5 262 L 935 212 Z" fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(955,200)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="65" height="22" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 67px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Yes/No</div></div></foreignObject><text x="33" y="21" fill="#000000" text-anchor="middle" font-size="20px" fo
 nt-family="Helvetica">[Not supported by viewer]</text></switch></g><rect x="742" y="182" width="120" height="60" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(743,187)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="116" height="49" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 116px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Schedule t<span>oplogy </span><i>t<sub>i</sub></i></div></div></foreignObject><text x="58" y="35" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 935 212 L 868.37 212" fill="none" stroke="#000000" stroke
 -miterlimit="10" pointer-events="none"/><path d="M 863.12 212 L 870.12 208.5 L 868.37 212 L 870.12 215.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1286 260 L 1286 293 L 1286.41 320.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1286.48 325.88 L 1282.88 318.93 L 1286.41 320.63 L 1289.88 318.83 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1145" y="165" width="280" height="95" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1146,189)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="276" height="46" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 276px; white-space: normal; text-align: center;"><
 div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Determine if topologies should be evicted</div></div></foreignObject><text x="138" y="33" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1040.93 213 L 1094 213 L 1138.63 213" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1143.88 213 L 1136.88 216.5 L 1138.63 213 L 1136.88 209.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 988 113 L 988 156.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 988 161.88 L 984.5 154.88 L 988 156.63 L 991.5 154.88 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><ellipse cx="989" cy="57" rx="113.5" ry="55" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(876,30)"><switch><f
 oreignObject style="overflow:visible;" pointer-events="all" width="223" height="52" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 223px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><font style="font-size: 20px">Can topology <i>t<sub>i </sub></i>from user <i>u<sub>i </sub></i>be scheduled?</font></div></div></foreignObject><text x="112" y="36" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1285.5 622 L 1346 681 L 1285.5 740 L 1225 681 Z" fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(1259,669)"><switch><foreignObject style="overflo
 w:visible;" pointer-events="all" width="51" height="22" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 51px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">&gt;0 / 0</div></div></foreignObject><text x="26" y="21" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1286.5 532 L 1286 577 L 1286 616.12" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1286 621.37 L 1282.5 614.37 L 1286 616.12 L 1289.5 614.37 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><ellipse cx="1287" cy="430" rx="262.5" ry="102.5" fill="#ffffff" stroke="#0000
 00" pointer-events="none"/><g transform="translate(1025,376)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="521" height="106" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 521px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><span>How much resources does topology </span><i>t</i><sub style="font-style: italic">i  </sub>need outside of <span>user </span><i>u</i><sub style="font-style: italic">i</sub>'s resource guarantee:<br /><div><i>u</i><sub>i</sub><span>'s </span>allocated resource above guarantee + additional resources needed above guarantee to schedule <i>t</i><sub>i</sub></div></div></div></foreignObject><text x="261" y="63" fill="#00
 0000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1346 681 L 1422 681 L 1444.63 681" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1449.88 681 L 1442.88 684.5 L 1444.63 681 L 1442.88 677.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1764" y="1002" width="205" height="94" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1765,1002)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="201" height="92" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 201px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:in
 herit;text-decoration:inherit;"><p>Determine if <i>t<sub>j </sub></i>is killed can we schedule <i>t<sub>i </sub>.</i></p></div></div></foreignObject><text x="101" y="56" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><rect x="1451" y="622" width="261" height="126" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1453,636)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="257" height="97" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 257px; white-space: normal;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><div style="text-align: center"><span style="line-height: 1.2">Find user </span>
 <i style="line-height: 1.2">u<sub>j</sub></i><span style="line-height: 1.2">, the user that has the most resources above his or her guarantee</span></div><div style="text-align: center"><br /></div></div></div></foreignObject><text x="129" y="59" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1866.5 1162 L 1866.5 1102.37" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1866.5 1097.12 L 1870 1104.12 L 1866.5 1102.37 L 1863 1104.12 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1813.54 1211 L 1472 1211 L 1138.37 1211.49" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1133.12 1211.5 L 1140.11 1207.99 L 1138.37 1211.49 L 1140.12 1214.99 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1866.5 1162 L 1920 1211.5 L 1866.5 1261 L 1813 1211.5 Z"
  fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(1833,1200)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="65" height="22" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 67px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><div style="text-align: center">Yes/No</div></div></div></foreignObject><text x="33" y="21" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 2118 1149 L 2118 685 L 1718.37 685" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1713.12 685 L 1720.12 681.5 L 1
 718.37 685 L 1720.12 688.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1996" y="1149" width="241" height="125" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1997,1130)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="237" height="161" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 237px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><p>Recalculate how much resources users have above their guarantee taken into account that <i>t</i><sub style="font-style: italic">j</sub>'s is killed.</p></div></div></foreignObject><text x="119" y="91" fill="#000000" text-anchor="middle" fo
 nt-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1920 1211.5 L 1989.63 1211.5" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1994.88 1211.5 L 1987.88 1215 L 1989.63 1211.5 L 1987.88 1208 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 766 808 L 766 835 L 766 857.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 766 862.88 L 762.5 855.88 L 766 857.63 L 769.5 855.88 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="632" y="554" width="269" height="254" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(633,581)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="265" height="199" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-siz
 e: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 265px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><div style="text-align: center"><span style="line-height: 1.2">Find user </span><i style="line-height: 1.2">u</i><sub style="font-style: italic">j</sub><span style="line-height: 1.2">, the user who has the most  resources above his or her resource guarantee &gt; </span><i>u</i><sub>i</sub><span>'s </span><span>allocated</span><span> resource above guarantee + additional resources needed above guarantee to schedule </span><i>t</i><sub>i</sub><br /></div></div></div></foreignObject><text x="133" y="110" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1225 681 L 907.37 681" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-eve
 nts="none"/><path d="M 902.12 681 L 909.12 677.5 L 907.37 681 L 909.12 684.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1286" y="1412" width="256" height="110" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1287,1443)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="252" height="46" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 252px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Topology cannot be scheduled. </div></div></foreignObject><text x="126" y="33" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></sw
 itch></g><path d="M 1866.5 949 L 1866.5 995.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1866.5 1000.88 L 1863 993.88 L 1866.5 995.63 L 1870 993.88 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1751" y="815" width="231" height="134" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1752,831)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="227" height="100" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 227px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><span>Find a topology </span><i>t<sub>j </sub></i><span>with the lowest prio
 rity that has been scheduled and belongs to user </span><i>u<sub>j</sub></i><span>.</span></div></div></foreignObject><text x="114" y="60" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1582 817 L 1645 882 L 1582 947 L 1519 882 Z" fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(1548,858)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="66" height="46" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 66px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">None?<div>Yes/No</div></div></div></foreignObject><text x="33"
  y="33" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1582 748 L 1582 783 L 1582 810.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1582 815.88 L 1578.5 808.88 L 1582 810.63 L 1585.5 808.88 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1645 882 L 1744.63 882" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1749.88 882 L 1742.88 885.5 L 1744.63 882 L 1742.88 878.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1368 957 L 1368 1185 L 1414 1185 L 1414 1406.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1414 1411.88 L 1410.5 1404.88 L 1414 1406.63 L 1417.5 1404.88 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1272" y="807" width="193" height="150"
  fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1273,822)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="189" height="118" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 189px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><span>Total cluster resources (+/- room for fragmention) &lt; Total resource guarantees</span></div></div></foreignObject><text x="95" y="69" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 835 933 L 868 933 L 894.63 933" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 899.88
  933 L 892.88 936.5 L 894.63 933 L 892.88 929.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 765 863 L 835 933 L 765 1003 L 695 933 Z" fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(731,909)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="66" height="46" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 66px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">None?<div>No/Yes</div></div></div></foreignObject><text x="33" y="33" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d
 ="M 986 1010 L 986 1467 L 1279.63 1467" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1284.88 1467 L 1277.88 1470.5 L 1279.63 1467 L 1277.88 1463.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="901" y="857" width="170" height="153" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(902,896)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="166" height="73" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 166px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Wont be fair to allocate <span>user </span><i>u</i><sub style="font-style: italic">i
  </sub>more resources</div></div></foreignObject><text x="83" y="47" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 478 1202 L 478 1239 L 478 1268.11" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 478 1273.36 L 474.5 1266.36 L 478 1268.11 L 481.5 1266.36 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="372" y="1092" width="210" height="110" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(373,1100)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="206" height="92" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 206px; white-space: normal; text-align: center;"><div x
 mlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><p>Determine if <i>t<sub>j </sub></i>is killed can we schedule <i>t<sub>i </sub>.</i></p></div></div></foreignObject><text x="103" y="56" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 415.53 1333 L 362 1333 L 314.37 1333" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 309.12 1333 L 316.12 1329.5 L 314.37 1333 L 316.12 1336.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 538.42 1335 L 1134 1335 L 1134 531 L 802 531 L 802 249.37" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 802 244.12 L 805.5 251.12 L 802 249.37 L 798.5 251.12 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 477.5 1274 L 540 1333.5 L 477.5 1393 L 415 1333.5 Z
 " fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(444,1322)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="65" height="22" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 67px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><div style="text-align: center">No/Yes</div></div></div></foreignObject><text x="33" y="21" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 154 1263 L 154 681 L 625.63 681" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 630.88 681 L 623.88 684.5 L 625.63
  681 L 623.88 677.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="2" y="1262" width="305" height="142" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(3,1264)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="301" height="137" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 301px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><p>Recalculate how much resources users have above their guarantee taken into account that <i>t</i><sub style="font-style: italic">j</sub>'s is killed.</p></div></div></foreignObject><text x="151" y="79" fill="#000000" text-anchor="middle" font-size="20px
 " font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 603 933 L 688.63 933" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 693.88 933 L 686.88 936.5 L 688.63 933 L 686.88 929.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 478 1010 L 478 1051 L 478 1085.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 478 1090.88 L 474.5 1083.88 L 478 1085.63 L 481.5 1083.88 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="353" y="856" width="250" height="154" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(354,882)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="246" height="100" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: 
 Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 246px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><span>Find a topology </span><i>t<sub>j </sub></i><span>with the lowest priority that has been scheduled and belongs to user </span><i>u</i><sub style="font-style: italic">j </sub></div></div></foreignObject><text x="123" y="60" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1519 882 L 1471.37 882" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1466.12 882 L 1473.12 878.5 L 1471.37 882 L 1473.12 885.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/></g></svg>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e45ccfe7/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
index ffdf460..d42915b 100644
--- a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
+++ b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
@@ -202,26 +202,26 @@ public class TestResourceAwareScheduler {
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
 
         TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
-        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
-        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 29);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 29);
         TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
-        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 29);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
 
         TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
-        TopologyDetails topo7 = TestUtilsForResourceAwareScheduler.getTopology("topo-7", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
-        TopologyDetails topo8 = TestUtilsForResourceAwareScheduler.getTopology("topo-8", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30);
+        TopologyDetails topo7 = TestUtilsForResourceAwareScheduler.getTopology("topo-7", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 29);
+        TopologyDetails topo8 = TestUtilsForResourceAwareScheduler.getTopology("topo-8", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 29);
         TopologyDetails topo9 = TestUtilsForResourceAwareScheduler.getTopology("topo-9", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
-        TopologyDetails topo10 = TestUtilsForResourceAwareScheduler.getTopology("topo-10", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30);
+        TopologyDetails topo10 = TestUtilsForResourceAwareScheduler.getTopology("topo-10", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 29);
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
         TopologyDetails topo11 = TestUtilsForResourceAwareScheduler.getTopology("topo-11", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
-        TopologyDetails topo12 = TestUtilsForResourceAwareScheduler.getTopology("topo-12", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
-        TopologyDetails topo13 = TestUtilsForResourceAwareScheduler.getTopology("topo-13", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 30);
+        TopologyDetails topo12 = TestUtilsForResourceAwareScheduler.getTopology("topo-12", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 29);
+        TopologyDetails topo13 = TestUtilsForResourceAwareScheduler.getTopology("topo-13", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 29);
         TopologyDetails topo14 = TestUtilsForResourceAwareScheduler.getTopology("topo-14", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
-        TopologyDetails topo15 = TestUtilsForResourceAwareScheduler.getTopology("topo-15", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30);
+        TopologyDetails topo15 = TestUtilsForResourceAwareScheduler.getTopology("topo-15", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 29);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo1.getId(), topo1);
@@ -288,7 +288,7 @@ public class TestResourceAwareScheduler {
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
 
         TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 5, 15, 1, 1, Time.currentTimeSecs() - 2, 20);
-        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 30);
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 5, 15, 1, 1, Time.currentTimeSecs() - 8, 29);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo1.getId(), topo1);
@@ -363,7 +363,7 @@ public class TestResourceAwareScheduler {
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
-        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo1.getId(), topo1);
@@ -477,8 +477,8 @@ public class TestResourceAwareScheduler {
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
-        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
-        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
 
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
@@ -587,8 +587,8 @@ public class TestResourceAwareScheduler {
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
-        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
-        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 15, 30);
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 15, 29);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo2.getId(), topo2);
@@ -759,7 +759,7 @@ public class TestResourceAwareScheduler {
 
         TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
         TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
-        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
         TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
 
 
@@ -769,7 +769,7 @@ public class TestResourceAwareScheduler {
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
-        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo1.getId(), topo1);
@@ -927,7 +927,7 @@ public class TestResourceAwareScheduler {
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
-        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
         TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
@@ -1079,7 +1079,7 @@ public class TestResourceAwareScheduler {
 
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
-        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
         TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
 
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();


[15/23] storm git commit: making edits based on review comments

Posted by da...@apache.org.
making edits based on review comments


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

Branch: refs/heads/master
Commit: a7f691350b435068f042ba1e9b83d1c317a4eaf7
Parents: 9a542a6
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Tue Dec 15 16:46:18 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Tue Dec 15 23:53:28 2015 -0600

----------------------------------------------------------------------
 conf/defaults.yaml                              |   2 +
 .../Resource_Aware_Scheduler_overview.md        |  17 +-
 .../starter/ResourceAwareExampleTopology.java   |   5 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  15 +-
 storm-core/src/jvm/backtype/storm/Config.java   |   2 +-
 .../jvm/backtype/storm/scheduler/Cluster.java   |  17 +-
 .../backtype/storm/scheduler/Topologies.java    |   4 +-
 .../storm/scheduler/TopologyDetails.java        |  76 +++----
 .../storm/scheduler/resource/RAS_Node.java      | 215 +++++++++----------
 .../storm/scheduler/resource/RAS_Nodes.java     |   5 +-
 .../resource/ResourceAwareScheduler.java        |  14 +-
 .../eviction/DefaultEvictionStrategy.java       |   1 -
 .../DefaultSchedulingPriorityStrategy.java      |   1 -
 .../DefaultResourceAwareStrategy.java           |   3 +-
 .../strategies/scheduling/IStrategy.java        |   4 +-
 .../storm/validation/ConfigValidation.java      |   3 +-
 .../resource/TestResourceAwareScheduler.java    |  10 -
 .../TestUtilsForResourceAwareScheduler.java     |   3 -
 18 files changed, 191 insertions(+), 206 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 448db46..1297e3e 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -251,6 +251,8 @@ topology.disruptor.batch.timeout.millis: 1
 topology.disable.loadaware: false
 
 # Configs for Resource Aware Scheduler
+# topology priority describing the importance of the topology in decreasing importance starting from 0 (i.e. 0 is the highest priority and the priority importance decreases as the priority number increases).
+# Recommended range of 0-29 but no hard limit set.
 topology.priority: 29
 topology.component.resources.onheap.memory.mb: 128.0
 topology.component.resources.offheap.memory.mb: 0.0

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/docs/documentation/Resource_Aware_Scheduler_overview.md
----------------------------------------------------------------------
diff --git a/docs/documentation/Resource_Aware_Scheduler_overview.md b/docs/documentation/Resource_Aware_Scheduler_overview.md
index f0ffd44..0f5b8cb 100644
--- a/docs/documentation/Resource_Aware_Scheduler_overview.md
+++ b/docs/documentation/Resource_Aware_Scheduler_overview.md
@@ -61,6 +61,8 @@ Example of Usage:
     s1.setCPULoad(15.0);
     builder.setBolt("exclaim1", new ExclamationBolt(), 3)
                 .shuffleGrouping("word").setCPULoad(10.0);
+    builder.setBolt("exclaim2", new HeavyBolt(), 1)
+                    .shuffleGrouping("exclaim1").setCPULoad(450.0);
 
 ###	Limiting the Heap Size per Worker (JVM) Process
 
@@ -70,7 +72,7 @@ Example of Usage:
 Parameters:
 * Number size – The memory limit a worker process will be allocated in megabytes
 
-The user can limit the amount of memory resources the resource aware scheduler that is allocated to a single worker on a per topology basis by using the above API.  This API is in place so that the users can spread executors to multiple workers.  However, spreading workers to multiple workers may increase the communication latency since executors will not be able to use Disruptor Queue for intra-process communication.
+The user can limit the amount of memory resources the resource aware scheduler allocates to a single worker on a per topology basis by using the above API.  This API is in place so that the users can spread executors to multiple workers.  However, spreading executors to multiple workers may increase the communication latency since executors will not be able to use Disruptor Queue for intra-process communication.
 
 Example of Usage:
 
@@ -116,7 +118,7 @@ The user can set some default configurations for the Resource Aware Scheduler in
 
 # Topology Priorities and Per User Resource 
 
-The next step for the Resource Aware Scheduler or RAS is to enable it to have multitenant capabilities since many Storm users typically share a Storm cluster.  Resource Aware Scheduler needs to be able to allocate resources on a per user basis.  Each user can be guaranteed a certain amount of resources to run his or her topologies and the Resource Aware Scheduler should meet those guarantees when possible.  When the Storm cluster has extra free resources, Resource Aware Scheduler needs to be able allocate additional resources to user in a fair manner. The importance of topologies can also vary.  Topologies can be used for actual production or just experimentation, thus Resource Aware Scheduler should take into account the importance of a topology when determining the order in which to schedule topologies or when to evict topologies
+The Resource Aware Scheduler or RAS also has multitenant capabilities since many Storm users typically share a Storm cluster.  Resource Aware Scheduler can allocate resources on a per user basis.  Each user can be guaranteed a certain amount of resources to run his or her topologies and the Resource Aware Scheduler will meet those guarantees when possible.  When the Storm cluster has extra free resources, Resource Aware Scheduler will to be able allocate additional resources to user in a fair manner. The importance of topologies can also vary.  Topologies can be used for actual production or just experimentation, thus Resource Aware Scheduler will take into account the importance of a topology when determining the order in which to schedule topologies or when to evict topologies
 
 ## Setup
 
@@ -158,9 +160,11 @@ Thus, each priority level contains 10 sub priorities. Users can set the priority
 Parameters:
 * priority – an integer representing the priority of the topology
 
+Please note that the 0-29 range is not a hard limit.  Thus, a user can set a priority number that is higher than 29. However, the property of higher the priority number, lower the importance still holds
+
 ### Specifying Scheduling Strategy:
 
-A user can specify on a per topology basis what scheduling strategy to use.  Users can implement the IStrategy interface and define a new strategies to schedule specific topologies.  This pluggable interface was created since we realize different topologies might have different scheduling needs.  A user can set the topology strategy within the topology definition by using the API:
+A user can specify on a per topology basis what scheduling strategy to use.  Users can implement the IStrategy interface and define new strategies to schedule specific topologies.  This pluggable interface was created since we realize different topologies might have different scheduling needs.  A user can set the topology strategy within the topology definition by using the API:
 
     public void setTopologyStrategy(Class<? extends IStrategy> clazz)
     
@@ -185,7 +189,7 @@ A default strategy will be provided.  The following explains how the default sch
 
 **DefaultSchedulingPriorityStrategy**
 
-The order of scheduling should be based on the distance a user’s current resource allocation to his or her guaranteed allocation.  We should prioritize the users who are the furthest away from their resource guarantee. The difficulty of this problem is that a user may have multiple resource guarantees, and another user can have another set resource guarantees, so how can we compare them in a fair manner?  Let's use the average percentage of resource guarantees satisfied as a method of comparison.
+The order of scheduling should be based on the distance between a user’s current resource allocation and his or her guaranteed allocation.  We should prioritize the users who are the furthest away from their resource guarantee. The difficulty of this problem is that a user may have multiple resource guarantees, and another user can have another set of resource guarantees, so how can we compare them in a fair manner?  Let's use the average percentage of resource guarantees satisfied as a method of comparison.
 
 For example:
 
@@ -204,10 +208,10 @@ User B’s average percentage satisfied of resource guarantee:
 
 Thus, in this example User A has a smaller average percentage of his or her resource guarantee satisfied than User B.  Thus, User A should get priority to be allocated more resource, i.e., schedule a topology submitted by User A.
 
-When scheduling, RAS sort users by the average percentage satisfied of resource guarantee and schedule topologies from users based on that ordering starting from the users with the lowest average percentage satisfied of resource guarantee.  When a user’s resource guarantee is completely satisfied, the user’s average percentage satisfied of resource guarantee will be greater than or equal to 1.
+When scheduling, RAS sorts users by the average percentage satisfied of resource guarantee and schedule topologies from users based on that ordering starting from the users with the lowest average percentage satisfied of resource guarantee.  When a user’s resource guarantee is completely satisfied, the user’s average percentage satisfied of resource guarantee will be greater than or equal to 1.
 
 ### Specifying Eviction Strategy
-The strategy for evicting topologies is also a pluggable interface in which the user can implement his or her own topology eviction strategy.  For a user to implement his or her own eviction strategy, he or she needs to implement the IEvictionStrategy Interface and set *Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY* to point to the implemented strategy class. For instance:
+The eviction strategy is used when there are not enough free resources in the cluster to schedule new topologies. If the cluster is full, we need a mechanism to evict topologies so that user resource guarantees can be met and additional resource can be shared fairly among users. The strategy for evicting topologies is also a pluggable interface in which the user can implement his or her own topology eviction strategy.  For a user to implement his or her own eviction strategy, he or she needs to implement the IEvictionStrategy Interface and set *Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY* to point to the implemented strategy class. For instance:
 
     resource.aware.scheduler.eviction.strategy: "backtype.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy"
 
@@ -215,7 +219,6 @@ A default eviction strategy is provided.  The following explains how the default
 
 **DefaultEvictionStrategy**
 
-If the cluster is full, we need a mechanism to evict topologies so that user resource guarantees can be met and additional resource can be shared fairly among users
 
 To determine if topology eviction should occur we should take into account the priority of the topology that we are trying to schedule and whether the resource guarantees for the owner of the topology have been met.  
 

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java b/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
index 7104281..0fb3724 100644
--- a/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
+++ b/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
@@ -82,8 +82,9 @@ public class ResourceAwareExampleTopology {
      */
     conf.setTopologyWorkerMaxHeapSize(1024.0);
 
-    // Set topology priority 0-30 with 0 being the highest priority and 30 being the lowest priority.
-    conf.setTopologyPriority(30);
+    //topology priority describing the importance of the topology in decreasing importance starting from 0 (i.e. 0 is the highest priority and the priority importance decreases as the priority number increases).
+    //Recommended range of 0-29 but no hard limit set.
+    conf.setTopologyPriority(29);
 
     // Set strategy to schedule topology. If not specified, default to backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy
     conf.setTopologyStrategy(backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class);

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index d76825d..0885ce3 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -533,24 +533,21 @@
       (.readBlob blob-store (master-stormconf-key storm-id) nimbus-subject))))
 
 (defn read-topology-details [nimbus storm-id]
-  (let [conf (:conf nimbus)
-        blob-store (:blob-store nimbus)
-        storm-base (.storm-base (:storm-cluster-state nimbus) storm-id nil)
+  (let [blob-store (:blob-store nimbus)
+        storm-base (or
+                     (.storm-base (:storm-cluster-state nimbus) storm-id nil)
+                     (throw (NotAliveException. storm-id)))
         topology-conf (read-storm-conf-as-nimbus storm-id blob-store)
         topology (read-storm-topology-as-nimbus storm-id blob-store)
         executor->component (->> (compute-executor->component nimbus storm-id)
                                  (map-key (fn [[start-task end-task]]
-                                            (ExecutorDetails. (int start-task) (int end-task)))))
-        launch-time-secs (if storm-base (:launch-time-secs storm-base)
-                           (throw
-                             (NotAliveException. (str storm-id))))]
+                                            (ExecutorDetails. (int start-task) (int end-task)))))]
     (TopologyDetails. storm-id
                       topology-conf
                       topology
                       (:num-workers storm-base)
                       executor->component
-                      launch-time-secs
-                      )))
+                      (:launch-time-secs storm-base))))
 
 ;; Does not assume that clocks are synchronized. Executor heartbeat is only used so that
 ;; nimbus knows when it's received a new heartbeat. All timing is done by nimbus and

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index dacf4f8..6c31c19 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -1861,7 +1861,7 @@ public class Config extends HashMap<String, Object> {
      * Sets the priority for a topology
      */
     @isInteger
-    @isPositiveNumber
+    @isPositiveNumber(includeZero = true)
     public static final String TOPOLOGY_PRIORITY = "topology.priority";
 
     /**

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
index 92b2219..53fdaa4 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
@@ -28,8 +28,6 @@ import java.util.Set;
 import backtype.storm.Config;
 import backtype.storm.networktopography.DNSToSwitchMapping;
 import backtype.storm.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class Cluster {
 
@@ -96,15 +94,15 @@ public class Cluster {
     /**
      * Get a copy of this cluster object
      */
-    public Cluster getCopy() {
+    public static Cluster getCopy(Cluster cluster) {
         HashMap<String, SchedulerAssignmentImpl> newAssignments = new HashMap<String, SchedulerAssignmentImpl>();
-        for (Map.Entry<String, SchedulerAssignmentImpl> entry : this.assignments.entrySet()) {
+        for (Map.Entry<String, SchedulerAssignmentImpl> entry : cluster.assignments.entrySet()) {
             newAssignments.put(entry.getKey(), new SchedulerAssignmentImpl(entry.getValue().getTopologyId(), entry.getValue().getExecutorToSlot()));
         }
         Map newConf = new HashMap<String, Object>();
-        newConf.putAll(this.conf);
-        Cluster copy = new Cluster(this.inimbus, this.supervisors, newAssignments, newConf);
-        copy.status = new HashMap<>(this.status);
+        newConf.putAll(cluster.conf);
+        Cluster copy = new Cluster(cluster.inimbus, cluster.supervisors, newAssignments, newConf);
+        copy.status = new HashMap<>(cluster.status);
         return copy;
     }
     
@@ -622,14 +620,12 @@ public class Cluster {
     /**
      * set scheduler status for a topology
      */
-    private static final Logger LOG = LoggerFactory
-            .getLogger(Cluster.class);
     public void setStatus(String topologyId, String status) {
         this.status.put(topologyId, status);
     }
 
     /**
-     * Get all schedule statuses
+     * Get all topology scheduler statuses
      */
     public Map<String, String> getStatusMap() {
         return this.status;
@@ -639,6 +635,7 @@ public class Cluster {
      * set scheduler status map
      */
     public void setStatusMap(Map<String, String> statusMap) {
+        this.status.clear();
         this.status.putAll(statusMap);
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
index b6fbd07..3c8f987 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
@@ -68,8 +68,8 @@ public class Topologies {
         return _allComponents;
     }
 
-    public Topologies getCopy() {
-        return new Topologies(this.topologies);
+    public static Topologies getCopy(Topologies topologies) {
+        return new Topologies(topologies.topologies);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
index 166493f..9e35981 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
@@ -45,7 +45,7 @@ public class TopologyDetails {
     private Map<ExecutorDetails, String> executorToComponent;
     private int numWorkers;
     //<ExecutorDetails - Task, Map<String - Type of resource, Map<String - type of that resource, Double - amount>>>
-    private Map<ExecutorDetails, Map<String, Double>> _resourceList;
+    private Map<ExecutorDetails, Map<String, Double>> resourceList;
     //Max heap size for a worker used by topology
     private Double topologyWorkerMaxHeapSize;
     //topology priority
@@ -56,31 +56,33 @@ public class TopologyDetails {
     private static final Logger LOG = LoggerFactory.getLogger(TopologyDetails.class);
 
     public TopologyDetails(String topologyId, Map topologyConf, StormTopology topology, int numWorkers) {
-        this.topologyId = topologyId;
-        this.topologyConf = topologyConf;
-        this.topology = topology;
-        this.numWorkers = numWorkers;
+        this(topologyId, topologyConf, topology,  numWorkers,  null, 0);
     }
 
     public TopologyDetails(String topologyId, Map topologyConf, StormTopology topology,
                            int numWorkers, Map<ExecutorDetails, String> executorToComponents) {
-        this(topologyId, topologyConf, topology, numWorkers);
+        this(topologyId, topologyConf, topology,  numWorkers,  executorToComponents, 0);
+    }
+
+    public TopologyDetails(String topologyId, Map topologyConf, StormTopology topology,
+                           int numWorkers, Map<ExecutorDetails, String> executorToComponents, int launchTime) {
+        this.topologyId = topologyId;
+        this.topologyConf = topologyConf;
+        this.topology = topology;
+        this.numWorkers = numWorkers;
         this.executorToComponent = new HashMap<>(0);
         if (executorToComponents != null) {
             this.executorToComponent.putAll(executorToComponents);
         }
-        this.initResourceList();
+        if (this.topology != null) {
+            this.initResourceList();
+        }
         this.initConfigs();
-    }
-
-    public TopologyDetails(String topologyId, Map topologyConf, StormTopology topology,
-                           int numWorkers, Map<ExecutorDetails, String> executorToComponents, int launchTime) {
-        this(topologyId, topologyConf, topology, numWorkers, executorToComponents);
         this.launchTime = launchTime;
     }
 
     public String getId() {
-        return topologyId;
+        return this.topologyId;
     }
 
     public String getName() {
@@ -88,11 +90,11 @@ public class TopologyDetails {
     }
 
     public Map getConf() {
-        return topologyConf;
+        return this.topologyConf;
     }
 
     public int getNumWorkers() {
-        return numWorkers;
+        return this.numWorkers;
     }
 
     public Map<ExecutorDetails, String> getExecutorToComponent() {
@@ -116,7 +118,7 @@ public class TopologyDetails {
     }
 
     private void initResourceList() {
-        _resourceList = new HashMap<>();
+        this.resourceList = new HashMap<>();
         // Extract bolt memory info
         if (this.topology.get_bolts() != null) {
             for (Map.Entry<String, Bolt> bolt : this.topology.get_bolts().entrySet()) {
@@ -124,9 +126,9 @@ public class TopologyDetails {
                 Map<String, Double> topology_resources = ResourceUtils.parseResources(bolt
                         .getValue().get_common().get_json_conf());
                 ResourceUtils.checkIntialization(topology_resources, bolt.getValue().toString(), this.topologyConf);
-                for (Map.Entry<ExecutorDetails, String> anExecutorToComponent : executorToComponent.entrySet()) {
+                for (Map.Entry<ExecutorDetails, String> anExecutorToComponent : this.executorToComponent.entrySet()) {
                     if (bolt.getKey().equals(anExecutorToComponent.getValue())) {
-                        _resourceList.put(anExecutorToComponent.getKey(), topology_resources);
+                        this.resourceList.put(anExecutorToComponent.getKey(), topology_resources);
                     }
                 }
             }
@@ -137,18 +139,18 @@ public class TopologyDetails {
                 Map<String, Double> topology_resources = ResourceUtils.parseResources(spout
                         .getValue().get_common().get_json_conf());
                 ResourceUtils.checkIntialization(topology_resources, spout.getValue().toString(), this.topologyConf);
-                for (Map.Entry<ExecutorDetails, String> anExecutorToComponent : executorToComponent.entrySet()) {
+                for (Map.Entry<ExecutorDetails, String> anExecutorToComponent : this.executorToComponent.entrySet()) {
                     if (spout.getKey().equals(anExecutorToComponent.getValue())) {
-                        _resourceList.put(anExecutorToComponent.getKey(), topology_resources);
+                        this.resourceList.put(anExecutorToComponent.getKey(), topology_resources);
                     }
                 }
             }
         } else {
-            LOG.warn("Topology " + topologyId + " does not seem to have any spouts!");
+            LOG.warn("Topology " + this.topologyId + " does not seem to have any spouts!");
         }
         //schedule tasks that are not part of components returned from topology.get_spout or topology.getbolt (AKA sys tasks most specifically __acker tasks)
         for(ExecutorDetails exec : this.getExecutors()) {
-            if (!_resourceList.containsKey(exec)) {
+            if (!this.resourceList.containsKey(exec)) {
                 LOG.debug(
                         "Scheduling {} {} with memory requirement as 'on heap' - {} and 'off heap' - {} and CPU requirement as {}",
                         this.getExecutorToComponent().get(exec),
@@ -163,7 +165,7 @@ public class TopologyDetails {
 
     private List<ExecutorDetails> componentToExecs(String comp) {
         List<ExecutorDetails> execs = new ArrayList<>();
-        for (Map.Entry<ExecutorDetails, String> entry : executorToComponent.entrySet()) {
+        for (Map.Entry<ExecutorDetails, String> entry : this.executorToComponent.entrySet()) {
             if (entry.getValue().equals(comp)) {
                 execs.add(entry.getKey());
             }
@@ -264,7 +266,7 @@ public class TopologyDetails {
     public Double getOnHeapMemoryRequirement(ExecutorDetails exec) {
         Double ret = null;
         if (hasExecInTopo(exec)) {
-            ret = _resourceList
+            ret = this.resourceList
                     .get(exec)
                     .get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB);
         }
@@ -281,7 +283,7 @@ public class TopologyDetails {
     public Double getOffHeapMemoryRequirement(ExecutorDetails exec) {
         Double ret = null;
         if (hasExecInTopo(exec)) {
-            ret = _resourceList
+            ret = this.resourceList
                     .get(exec)
                     .get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB);
         }
@@ -310,7 +312,7 @@ public class TopologyDetails {
      */
     public Map<ExecutorDetails, Double> getTotalMemoryResourceList() {
         Map<ExecutorDetails, Double> ret = new HashMap<>();
-        for (ExecutorDetails exec : _resourceList.keySet()) {
+        for (ExecutorDetails exec : this.resourceList.keySet()) {
             ret.put(exec, getTotalMemReqTask(exec));
         }
         return ret;
@@ -322,7 +324,7 @@ public class TopologyDetails {
      */
     public Double getTotalCpuReqTask(ExecutorDetails exec) {
         if (hasExecInTopo(exec)) {
-            return _resourceList
+            return this.resourceList
                     .get(exec)
                     .get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT);
         }
@@ -387,17 +389,17 @@ public class TopologyDetails {
      */
     public Map<String, Double> getTaskResourceReqList(ExecutorDetails exec) {
         if (hasExecInTopo(exec)) {
-            return _resourceList.get(exec);
+            return this.resourceList.get(exec);
         }
         return null;
     }
 
     /**
      * Checks if a executor is part of this topology
-     * @return Boolean whether or not a certain ExecutorDetail is included in the _resourceList.
+     * @return Boolean whether or not a certain ExecutorDetail is included in the resourceList.
      */
     public boolean hasExecInTopo(ExecutorDetails exec) {
-        return _resourceList != null && _resourceList.containsKey(exec);
+        return this.resourceList != null && this.resourceList.containsKey(exec);
     }
 
     /**
@@ -408,22 +410,22 @@ public class TopologyDetails {
             LOG.warn("Executor {} already exists...ResourceList: {}", exec, getTaskResourceReqList(exec));
             return;
         }
-        _resourceList.put(exec, resourceList);
+        this.resourceList.put(exec, resourceList);
     }
 
     /**
      * Add default resource requirements for a executor
      */
     public void addDefaultResforExec(ExecutorDetails exec) {
-        Double topologyComponentCpuPcorePercent = Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT), null);
+        Double topologyComponentCpuPcorePercent = Utils.getDouble(this.topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT), null);
         if (topologyComponentCpuPcorePercent == null) {
             LOG.warn("default value for topology.component.cpu.pcore.percent needs to be set!");
         }
-        Double topologyComponentResourcesOffheapMemoryMb = Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB), null);
+        Double topologyComponentResourcesOffheapMemoryMb = Utils.getDouble(this.topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB), null);
         if (topologyComponentResourcesOffheapMemoryMb == null) {
             LOG.warn("default value for topology.component.resources.offheap.memory.mb needs to be set!");
         }
-        Double topologyComponentResourcesOnheapMemoryMb = Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB), null);
+        Double topologyComponentResourcesOnheapMemoryMb = Utils.getDouble(this.topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB), null);
         if (topologyComponentResourcesOnheapMemoryMb == null) {
             LOG.warn("default value for topology.component.resources.onheap.memory.mb needs to be set!");
         }
@@ -434,9 +436,9 @@ public class TopologyDetails {
         defaultResourceList.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, topologyComponentResourcesOnheapMemoryMb);
         LOG.debug("Scheduling Executor: {} with memory requirement as onHeap: {} - offHeap: {} " +
                         "and CPU requirement: {}",
-                exec, topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB),
-                topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB),
-                topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT));
+                exec, this.topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB),
+                this.topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB),
+                this.topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT));
         addResourcesForExec(exec, defaultResourceList);
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
index 8d0df62..be39d2a 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
@@ -22,13 +22,11 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.Map.Entry;
 import java.util.ArrayList;
 
-import backtype.storm.Config;
 import backtype.storm.scheduler.Topologies;
 import backtype.storm.scheduler.TopologyDetails;
 import org.slf4j.Logger;
@@ -45,79 +43,76 @@ import backtype.storm.scheduler.WorkerSlot;
  */
 public class RAS_Node {
     private static final Logger LOG = LoggerFactory.getLogger(RAS_Node.class);
-    private Map<String, Set<WorkerSlot>> _topIdToUsedSlots = new HashMap<String, Set<WorkerSlot>>();
-    private Set<WorkerSlot> _freeSlots = new HashSet<WorkerSlot>();
-    private final String _nodeId;
-    private String _hostname;
-    private boolean _isAlive;
-    private SupervisorDetails _sup;
-    private Double _availMemory;
-    private Double _availCPU;
-    private List<WorkerSlot> _slots;
-    private Cluster _cluster;
-    private Topologies _topologies;
+    private Map<String, Set<WorkerSlot>> topIdToUsedSlots = new HashMap<String, Set<WorkerSlot>>();
+    private Set<WorkerSlot> freeSlots = new HashSet<WorkerSlot>();
+    private final String nodeId;
+    private String hostname;
+    private boolean isAlive;
+    private SupervisorDetails sup;
+    private Double availMemory;
+    private Double availCPU;
+    private Cluster cluster;
+    private Topologies topologies;
 
     public RAS_Node(String nodeId, Set<Integer> allPorts, boolean isAlive,
                     SupervisorDetails sup, Cluster cluster, Topologies topologies) {
-        _slots = new ArrayList<WorkerSlot>();
-        _nodeId = nodeId;
-        _isAlive = isAlive;
-        if (_isAlive && allPorts != null) {
+        this.nodeId = nodeId;
+        this.isAlive = isAlive;
+        if (this.isAlive && allPorts != null) {
             for (int port : allPorts) {
-                _freeSlots.add(new WorkerSlot(_nodeId, port));
+                this.freeSlots.add(new WorkerSlot(this.nodeId, port));
             }
-            _sup = sup;
-            _hostname = sup.getHost();
-            _availMemory = this.getTotalMemoryResources();
-            _availCPU = this.getTotalCpuResources();
-            _slots.addAll(_freeSlots);
+            this.sup = sup;
+            this.hostname = sup.getHost();
+            this.availMemory = getTotalMemoryResources();
+            this.availCPU = getTotalCpuResources();
         }
-        this._cluster = cluster;
-        this._topologies = topologies;
+        this.cluster = cluster;
+        this.topologies = topologies;
     }
 
     public String getId() {
-        return _nodeId;
+        return this.nodeId;
     }
 
     public String getHostname() {
-        return _hostname;
+        return this.hostname;
     }
 
     public Collection<WorkerSlot> getFreeSlots() {
-        return _freeSlots;
+        return this.freeSlots;
     }
 
     public Collection<WorkerSlot> getUsedSlots() {
         Collection<WorkerSlot> ret = new LinkedList<WorkerSlot>();
-        for (Collection<WorkerSlot> workers : _topIdToUsedSlots.values()) {
+        for (Collection<WorkerSlot> workers : this.topIdToUsedSlots.values()) {
             ret.addAll(workers);
         }
         return ret;
     }
 
     public boolean isAlive() {
-        return _isAlive;
+        return this.isAlive;
     }
 
     /**
      * @return a collection of the topology ids currently running on this node
      */
     public Collection<String> getRunningTopologies() {
-        return _topIdToUsedSlots.keySet();
+        return this.topIdToUsedSlots.keySet();
     }
 
     public boolean isTotallyFree() {
-        return _topIdToUsedSlots.isEmpty();
+        return this.topIdToUsedSlots.isEmpty();
     }
 
     public int totalSlotsFree() {
-        return _freeSlots.size();
+        return this.freeSlots.size();
     }
 
     public int totalSlotsUsed() {
         int total = 0;
-        for (Set<WorkerSlot> slots : _topIdToUsedSlots.values()) {
+        for (Set<WorkerSlot> slots : this.topIdToUsedSlots.values()) {
             total += slots.size();
         }
         return total;
@@ -129,7 +124,7 @@ public class RAS_Node {
 
     public int totalSlotsUsed(String topId) {
         int total = 0;
-        Set<WorkerSlot> slots = _topIdToUsedSlots.get(topId);
+        Set<WorkerSlot> slots = this.topIdToUsedSlots.get(topId);
         if (slots != null) {
             total = slots.size();
         }
@@ -137,42 +132,42 @@ public class RAS_Node {
     }
 
     private void validateSlot(WorkerSlot ws) {
-        if (!_nodeId.equals(ws.getNodeId())) {
+        if (!this.nodeId.equals(ws.getNodeId())) {
             throw new IllegalArgumentException(
                     "Trying to add a slot to the wrong node " + ws +
-                            " is not a part of " + _nodeId);
+                            " is not a part of " + this.nodeId);
         }
     }
 
     void addOrphanedSlot(WorkerSlot ws) {
-        if (_isAlive) {
+        if (this.isAlive) {
             throw new IllegalArgumentException("Orphaned Slots " +
                     "only are allowed on dead nodes.");
         }
         validateSlot(ws);
-        if (_freeSlots.contains(ws)) {
+        if (this.freeSlots.contains(ws)) {
             return;
         }
-        for (Set<WorkerSlot> used : _topIdToUsedSlots.values()) {
+        for (Set<WorkerSlot> used : this.topIdToUsedSlots.values()) {
             if (used.contains(ws)) {
                 return;
             }
         }
-        _freeSlots.add(ws);
+        this.freeSlots.add(ws);
     }
 
     boolean assignInternal(WorkerSlot ws, String topId, boolean dontThrow) {
         validateSlot(ws);
-        if (!_freeSlots.remove(ws)) {
+        if (!this.freeSlots.remove(ws)) {
             if (dontThrow) {
                 return true;
             }
             throw new IllegalStateException("Assigning a slot that was not free " + ws);
         }
-        Set<WorkerSlot> usedSlots = _topIdToUsedSlots.get(topId);
+        Set<WorkerSlot> usedSlots = this.topIdToUsedSlots.get(topId);
         if (usedSlots == null) {
             usedSlots = new HashSet<WorkerSlot>();
-            _topIdToUsedSlots.put(topId, usedSlots);
+            this.topIdToUsedSlots.put(topId, usedSlots);
         }
         usedSlots.add(ws);
         return false;
@@ -182,18 +177,18 @@ public class RAS_Node {
      * Free all slots on this node.  This will update the Cluster too.
      */
     public void freeAllSlots() {
-        if (!_isAlive) {
-            LOG.warn("Freeing all slots on a dead node {} ", _nodeId);
-        }
-        for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
-            _cluster.freeSlots(entry.getValue());
-            _availCPU = this.getTotalCpuResources();
-            _availMemory = this.getAvailableMemoryResources();
-            if (_isAlive) {
-                _freeSlots.addAll(entry.getValue());
+        if (!this.isAlive) {
+            LOG.warn("Freeing all slots on a dead node {} ", this.nodeId);
+        }
+        for (Entry<String, Set<WorkerSlot>> entry : this.topIdToUsedSlots.entrySet()) {
+            this.cluster.freeSlots(entry.getValue());
+            this.availCPU = this.getTotalCpuResources();
+            this.availMemory = this.getAvailableMemoryResources();
+            if (this.isAlive) {
+                this.freeSlots.addAll(entry.getValue());
             }
         }
-        _topIdToUsedSlots = new HashMap<String, Set<WorkerSlot>>();
+        this.topIdToUsedSlots = new HashMap<String, Set<WorkerSlot>>();
     }
 
     /**
@@ -201,16 +196,16 @@ public class RAS_Node {
      * @param ws the slot to free
      */
     public void free(WorkerSlot ws) {
-        LOG.info("freeing ws {} on node {}", ws, _hostname);
-        if (_freeSlots.contains(ws)) return;
-        for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
+        LOG.info("freeing ws {} on node {}", ws, this.hostname);
+        if (this.freeSlots.contains(ws)) return;
+        for (Entry<String, Set<WorkerSlot>> entry : this.topIdToUsedSlots.entrySet()) {
             Set<WorkerSlot> slots = entry.getValue();
             double memUsed = this.getMemoryUsedByWorker(ws);
             double cpuUsed = this.getCpuUsedByWorker(ws);
             if (slots.remove(ws)) {
-                _cluster.freeSlot(ws);
-                if (_isAlive) {
-                    _freeSlots.add(ws);
+                this.cluster.freeSlot(ws);
+                if (this.isAlive) {
+                    this.freeSlots.add(ws);
                 }
                 this.freeMemory(memUsed);
                 this.freeCPU(cpuUsed);
@@ -218,7 +213,7 @@ public class RAS_Node {
             }
         }
         throw new IllegalArgumentException("Tried to free a slot that was not" +
-                " part of this node " + _nodeId);
+                " part of this node " + this.nodeId);
     }
 
     /**
@@ -226,35 +221,37 @@ public class RAS_Node {
      * @param topId the topology to free slots for
      */
     public void freeTopology(String topId) {
-        Set<WorkerSlot> slots = _topIdToUsedSlots.get(topId);
+        Set<WorkerSlot> slots = this.topIdToUsedSlots.get(topId);
         if (slots == null || slots.isEmpty()) {
             return;
         }
         for (WorkerSlot ws : slots) {
-            _cluster.freeSlot(ws);
+            this.cluster.freeSlot(ws);
             this.freeMemory(this.getMemoryUsedByWorker(ws));
             this.freeCPU(this.getCpuUsedByWorker(ws));
-            if (_isAlive) {
-                _freeSlots.add(ws);
+            if (this.isAlive) {
+                this.freeSlots.add(ws);
             }
         }
-        _topIdToUsedSlots.remove(topId);
+        this.topIdToUsedSlots.remove(topId);
     }
 
     private void freeMemory(double amount) {
-        _availMemory += amount;
-        LOG.debug("freeing {} memory on node {}...avail mem: {}", amount, this.getHostname(), _availMemory);
-        if (_availMemory > this.getTotalMemoryResources()) {
+        LOG.debug("freeing {} memory on node {}...avail mem: {}", amount, this.getHostname(), this.availMemory);
+        if((this.availMemory + amount) > this.getTotalCpuResources()) {
             LOG.warn("Freeing more memory than there exists!");
+            return;
         }
+        this.availMemory += amount;
     }
 
     private void freeCPU(double amount) {
-        _availCPU += amount;
-        LOG.debug("freeing {} CPU on node...avail CPU: {}", amount, this.getHostname(), _availCPU);
-        if (_availCPU > this.getAvailableCpuResources()) {
-            LOG.warn("Freeing more memory than there exists!");
+        LOG.debug("freeing {} CPU on node...avail CPU: {}", amount, this.getHostname(), this.availCPU);
+        if ((this.availCPU + amount) > this.getAvailableCpuResources()) {
+            LOG.warn("Freeing more CPU than there exists!");
+            return;
         }
+        this.availCPU += amount;
     }
 
     public double getMemoryUsedByWorker(WorkerSlot ws) {
@@ -262,7 +259,7 @@ public class RAS_Node {
         if (topo == null) {
             return 0.0;
         }
-        Collection<ExecutorDetails> execs = this.getExecutors(ws, this._cluster);
+        Collection<ExecutorDetails> execs = this.getExecutors(ws, this.cluster);
         double totalMemoryUsed = 0.0;
         for (ExecutorDetails exec : execs) {
             totalMemoryUsed += topo.getTotalMemReqTask(exec);
@@ -275,7 +272,7 @@ public class RAS_Node {
         if (topo == null) {
             return 0.0;
         }
-        Collection<ExecutorDetails> execs = this.getExecutors(ws, this._cluster);
+        Collection<ExecutorDetails> execs = this.getExecutors(ws, this.cluster);
         double totalCpuUsed = 0.0;
         for (ExecutorDetails exec : execs) {
             totalCpuUsed += topo.getTotalCpuReqTask(exec);
@@ -284,12 +281,12 @@ public class RAS_Node {
     }
 
     public TopologyDetails findTopologyUsingWorker(WorkerSlot ws) {
-        for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
+        for (Entry<String, Set<WorkerSlot>> entry : this.topIdToUsedSlots.entrySet()) {
             String topoId = entry.getKey();
             Set<WorkerSlot> workers = entry.getValue();
             for (WorkerSlot worker : workers) {
                 if (worker.getNodeId().equals(ws.getNodeId()) && worker.getPort() == ws.getPort()) {
-                    return _topologies.getById(topoId);
+                    return this.topologies.getById(topoId);
                 }
             }
         }
@@ -324,24 +321,24 @@ public class RAS_Node {
     }
 
     public void assign(WorkerSlot target, TopologyDetails td, Collection<ExecutorDetails> executors) {
-        if (!_isAlive) {
-            throw new IllegalStateException("Trying to adding to a dead node " + _nodeId);
+        if (!this.isAlive) {
+            throw new IllegalStateException("Trying to adding to a dead node " + this.nodeId);
         }
-        if (_freeSlots.isEmpty()) {
-            throw new IllegalStateException("Trying to assign to a full node " + _nodeId);
+        if (this.freeSlots.isEmpty()) {
+            throw new IllegalStateException("Trying to assign to a full node " + this.nodeId);
         }
         if (executors.size() == 0) {
-            LOG.warn("Trying to assign nothing from " + td.getId() + " to " + _nodeId + " (Ignored)");
+            LOG.warn("Trying to assign nothing from " + td.getId() + " to " + this.nodeId + " (Ignored)");
         }
 
         if (target == null) {
-            target = _freeSlots.iterator().next();
+            target = this.freeSlots.iterator().next();
         }
-        if (!_freeSlots.contains(target)) {
-            throw new IllegalStateException("Trying to assign already used slot" + target.getPort() + "on node " + _nodeId);
+        if (!this.freeSlots.contains(target)) {
+            throw new IllegalStateException("Trying to assign already used slot" + target.getPort() + "on node " + this.nodeId);
         } else {
             allocateResourceToSlot(td, executors, target);
-            _cluster.assign(target, td.getId(), executors);
+            this.cluster.assign(target, td.getId(), executors);
             assignInternal(target, td.getId(), false);
         }
     }
@@ -359,21 +356,21 @@ public class RAS_Node {
     @Override
     public boolean equals(Object other) {
         if (other instanceof RAS_Node) {
-            return _nodeId.equals(((RAS_Node) other)._nodeId);
+            return this.nodeId.equals(((RAS_Node) other).nodeId);
         }
         return false;
     }
 
     @Override
     public int hashCode() {
-        return _nodeId.hashCode();
+        return this.nodeId.hashCode();
     }
 
     @Override
     public String toString() {
-        return "{Node: " + ((_sup == null) ? "null (possibly down)" : _sup.getHost())
-                + ", AvailMem: " + ((_availMemory == null) ? "N/A" : _availMemory.toString())
-                + ", AvailCPU: " + ((_availCPU == null) ? "N/A" : _availCPU.toString()) + "}";
+        return "{Node: " + ((this.sup == null) ? "null (possibly down)" : this.sup.getHost())
+                + ", AvailMem: " + ((this.availMemory == null) ? "N/A" : this.availMemory.toString())
+                + ", this.availCPU: " + ((this.availCPU == null) ? "N/A" : this.availCPU.toString()) + "}";
     }
 
     public static int countSlotsUsed(String topId, Collection<RAS_Node> nodes) {
@@ -436,7 +433,7 @@ public class RAS_Node {
      * @param amount the amount to set as available memory
      */
     public void setAvailableMemory(Double amount) {
-        _availMemory = amount;
+        this.availMemory = amount;
     }
 
     /**
@@ -444,10 +441,10 @@ public class RAS_Node {
      * @return the available memory for this node
      */
     public Double getAvailableMemoryResources() {
-        if (_availMemory == null) {
+        if (this.availMemory == null) {
             return 0.0;
         }
-        return _availMemory;
+        return this.availMemory;
     }
 
     /**
@@ -455,8 +452,8 @@ public class RAS_Node {
      * @return the total memory for this node
      */
     public Double getTotalMemoryResources() {
-        if (_sup != null && _sup.getTotalMemory() != null) {
-            return _sup.getTotalMemory();
+        if (this.sup != null && this.sup.getTotalMemory() != null) {
+            return this.sup.getTotalMemory();
         } else {
             return 0.0;
         }
@@ -468,12 +465,12 @@ public class RAS_Node {
      * @return the current available memory for this node after consumption
      */
     public Double consumeMemory(Double amount) {
-        if (amount > _availMemory) {
-            LOG.error("Attempting to consume more memory than available! Needed: {}, we only have: {}", amount, _availMemory);
+        if (amount > this.availMemory) {
+            LOG.error("Attempting to consume more memory than available! Needed: {}, we only have: {}", amount, this.availMemory);
             return null;
         }
-        _availMemory = _availMemory - amount;
-        return _availMemory;
+        this.availMemory = this.availMemory - amount;
+        return this.availMemory;
     }
 
     /**
@@ -481,10 +478,10 @@ public class RAS_Node {
      * @return the available cpu for this node
      */
     public Double getAvailableCpuResources() {
-        if (_availCPU == null) {
+        if (this.availCPU == null) {
             return 0.0;
         }
-        return _availCPU;
+        return this.availCPU;
     }
 
     /**
@@ -492,8 +489,8 @@ public class RAS_Node {
      * @return the total cpu for this node
      */
     public Double getTotalCpuResources() {
-        if (_sup != null && _sup.getTotalCPU() != null) {
-            return _sup.getTotalCPU();
+        if (this.sup != null && this.sup.getTotalCPU() != null) {
+            return this.sup.getTotalCPU();
         } else {
             return 0.0;
         }
@@ -505,12 +502,12 @@ public class RAS_Node {
      * @return the current available cpu for this node after consumption
      */
     public Double consumeCPU(Double amount) {
-        if (amount > _availCPU) {
-            LOG.error("Attempting to consume more CPU than available! Needed: {}, we only have: {}", amount, _availCPU);
+        if (amount > this.availCPU) {
+            LOG.error("Attempting to consume more CPU than available! Needed: {}, we only have: {}", amount, this.availCPU);
             return null;
         }
-        _availCPU = _availCPU - amount;
-        return _availCPU;
+        this.availCPU = this.availCPU - amount;
+        return this.availCPU;
     }
 
     /**
@@ -526,6 +523,6 @@ public class RAS_Node {
     }
 
     public Map<String, Set<WorkerSlot>> getTopoIdTousedSlots() {
-        return _topIdToUsedSlots;
+        return this.topIdToUsedSlots;
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
index 5a99bdd..abba0b0 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
@@ -65,7 +65,7 @@ public class RAS_Nodes {
                     nodeIdToNode.put(id, node);
                 }
                 if (!node.isAlive()) {
-                    //The supervisor on the node down so add an orphaned slot to hold the unsupervised worker
+                    //The supervisor on the node is down so add an orphaned slot to hold the unsupervised worker
                     node.addOrphanedSlot(workerSlot);
                 }
                 if (node.assignInternal(workerSlot, topId, true)) {
@@ -113,7 +113,8 @@ public class RAS_Nodes {
                     if (topoMemoryResourceList.containsKey(exec)) {
                         node.consumeResourcesforTask(exec, topologies.getById(entry.getKey()));
                     } else {
-                        LOG.warn("Resource Req not found...Scheduling Task{} with memory requirement as on heap - {} and off heap - {} and CPU requirement as {}",
+                        LOG.warn("Resource Req not found...Scheduling Task {} with memory requirement as on heap - {} " +
+                                        "and off heap - {} and CPU requirement as {}",
                                 exec,
                                 Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB,
                                 Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT);

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
index c2e2fcd..0558e12 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
@@ -60,15 +60,13 @@ public class ResourceAwareScheduler implements IScheduler {
                 User user = userMapEntry.getValue();
                 this.userMap.put(userId, user.getCopy());
             }
-            this.cluster = cluster.getCopy();
-            this.topologies = topologies.getCopy();
+            this.cluster = Cluster.getCopy(cluster);
+            this.topologies = topologies.getCopy(topologies);
             this.nodes = new RAS_Nodes(this.cluster, this.topologies);
             this.conf.putAll(conf);
-
         }
     }
 
-
     @SuppressWarnings("rawtypes")
     private Map conf;
 
@@ -83,7 +81,7 @@ public class ResourceAwareScheduler implements IScheduler {
 
     @Override
     public void schedule(Topologies topologies, Cluster cluster) {
-        LOG.info("\n\n\nRerunning ResourceAwareScheduler...");
+        LOG.debug("\n\n\nRerunning ResourceAwareScheduler...");
         //initialize data structures
         this.initialize(topologies, cluster);
         //logs everything that is currently scheduled and the location at which they are scheduled
@@ -114,7 +112,7 @@ public class ResourceAwareScheduler implements IScheduler {
                 //Call scheduling priority strategy
                 td = schedulingPrioritystrategy.getNextTopologyToSchedule();
             } catch (Exception e) {
-                LOG.error("Exception thrown when running priority strategy {}. No topologies will be scheduled! Error: {} StackTrack: {}"
+                LOG.error("Exception thrown when running priority strategy {}. No topologies will be scheduled! Error: {} StackTrace: {}"
                         , schedulingPrioritystrategy.getClass().getName(), e.getMessage(), Arrays.toString(e.getStackTrace()));
                 break;
             }
@@ -153,7 +151,7 @@ public class ResourceAwareScheduler implements IScheduler {
                     rasStrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
                     result = rasStrategy.schedule(td);
                 } catch (Exception e) {
-                    LOG.error("Exception thrown when running strategy {} to schedule topology {}. Topology will not be scheduled! Error: {} StackTrack: {}"
+                    LOG.error("Exception thrown when running strategy {} to schedule topology {}. Topology will not be scheduled! Error: {} StackTrace: {}"
                             , rasStrategy.getClass().getName(), td.getName(), e.getMessage(), Arrays.toString(e.getStackTrace()));
                     this.restoreCheckpointSchedulingState(schedulingState);
                     //since state is restored need the update User topologySubmitter to the new User object in userMap
@@ -205,7 +203,7 @@ public class ResourceAwareScheduler implements IScheduler {
                                 evictionStrategy.prepare(this.topologies, this.cluster, this.userMap, this.nodes);
                                 madeSpace = evictionStrategy.makeSpaceForTopo(td);
                             } catch (Exception e) {
-                                LOG.error("Exception thrown when running eviction strategy {} to schedule topology {}. No evictions will be done! Error: {} StackTrack: {}"
+                                LOG.error("Exception thrown when running eviction strategy {} to schedule topology {}. No evictions will be done! Error: {} StackTrace: {}"
                                         , evictionStrategy.getClass().getName(), td.getName(), e.getClass().getName(), Arrays.toString(e.getStackTrace()));
                                 this.restoreCheckpointSchedulingState(schedulingState);
                                 //since state is restored need the update User topologySubmitter to the new User object in userMap

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
index 81c2abc..1812580 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/eviction/DefaultEvictionStrategy.java
@@ -61,7 +61,6 @@ public class DefaultEvictionStrategy implements IEvictionStrategy {
         //user has enough resource under his or her resource guarantee to schedule topology
         if ((1.0 - submitter.getCPUResourcePoolUtilization()) >= cpuNeeded && (1.0 - submitter.getMemoryResourcePoolUtilization()) >= memoryNeeded) {
             if (evictUser != null) {
-
                 TopologyDetails topologyEvict = evictUser.getRunningTopologyWithLowestPriority();
                 evictTopology(topologyEvict);
                 return true;

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java
index 990ccd6..0d891ff 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/priority/DefaultSchedulingPriorityStrategy.java
@@ -60,7 +60,6 @@ public class DefaultSchedulingPriorityStrategy implements ISchedulingPriorityStr
         for (User user : this.userMap.values()) {
             if (user.hasTopologyNeedSchedule()) {
                 Double userResourcePoolAverageUtilization = user.getResourcePoolAverageUtilization();
-
                 if (least > userResourcePoolAverageUtilization) {
                     ret = user;
                     least = userResourcePoolAverageUtilization;

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/DefaultResourceAwareStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/DefaultResourceAwareStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/DefaultResourceAwareStrategy.java
index 75cc5eb..df4ae14 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/DefaultResourceAwareStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/DefaultResourceAwareStrategy.java
@@ -170,7 +170,8 @@ public class DefaultResourceAwareStrategy implements IStrategy {
             LOG.error("Not all executors successfully scheduled: {}",
                     executorsNotScheduled);
             schedulerAssignmentMap = null;
-            result = SchedulingResult.failure(SchedulingStatus.FAIL_NOT_ENOUGH_RESOURCES, (td.getExecutors().size() - unassignedExecutors.size()) + "/" + td.getExecutors().size() + " executors scheduled");
+            result = SchedulingResult.failure(SchedulingStatus.FAIL_NOT_ENOUGH_RESOURCES,
+                    (td.getExecutors().size() - unassignedExecutors.size()) + "/" + td.getExecutors().size() + " executors scheduled");
         } else {
             LOG.debug("All resources successfully scheduled!");
             result = SchedulingResult.successWithMsg(schedulerAssignmentMap, "Fully Scheduled by DefaultResourceAwareStrategy");

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java
index bb2e955..a7ac5c9 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/scheduling/IStrategy.java
@@ -47,8 +47,8 @@ public interface IStrategy {
      * @param td
      * @return returns a SchedulingResult object containing SchedulingStatus object to indicate whether scheduling is successful
      * The strategy must calculate a scheduling in the format of Map<WorkerSlot, Collection<ExecutorDetails>> where the key of
-     * this map is the worker slot that the value (collection of executors) should be assigned to. if a scheduling is calculated successfully,
-     * put the scheduling map in the SchedulingResult object.
+     * this map is the worker slot that the value (collection of executors) should be assigned to.
+     * if a scheduling is calculated successfully, put the scheduling map in the SchedulingResult object.
      */
     public SchedulingResult schedule(TopologyDetails td);
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java b/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
index 57c9f40..6df0108 100644
--- a/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
+++ b/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
@@ -540,7 +540,8 @@ public class ConfigValidation {
             try {
                 Class objectClass = Class.forName((String) o);
                 if (!this.classImplements.isAssignableFrom(objectClass)) {
-                    throw new IllegalArgumentException("Field " + name + " with value " + o + " does not implement " + this.classImplements.getName());
+                    throw new IllegalArgumentException("Field " + name + " with value " + o
+                            + " does not implement " + this.classImplements.getName());
                 }
             } catch (ClassNotFoundException e) {
                 throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
index d42915b..f9b4cd6 100644
--- a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
+++ b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
@@ -95,7 +95,6 @@ public class TestResourceAwareScheduler {
         TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 5, 15, 1, 1, Time.currentTimeSecs() - 16, 20);
         TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 5, 15, 1, 1, Time.currentTimeSecs() - 24, 30);
 
-
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo1.getId(), topo1);
         topoMap.put(topo2.getId(), topo2);
@@ -360,7 +359,6 @@ public class TestResourceAwareScheduler {
         TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
         TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
 
-
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
         TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
@@ -468,19 +466,16 @@ public class TestResourceAwareScheduler {
 
         TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 2, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
 
-
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
 
         TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
         TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
 
-
         config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
 
         TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
         TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 29);
 
-
         Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
         topoMap.put(topo2.getId(), topo2);
         topoMap.put(topo3.getId(), topo3);
@@ -747,7 +742,6 @@ public class TestResourceAwareScheduler {
         resourceUserPool.get("bobby").put("cpu", 100.0);
         resourceUserPool.get("bobby").put("memory", 1000.0);
 
-
         resourceUserPool.put("derek", new HashMap<String, Number>());
         resourceUserPool.get("derek").put("cpu", 100.0);
         resourceUserPool.get("derek").put("memory", 1000.0);
@@ -907,7 +901,6 @@ public class TestResourceAwareScheduler {
         resourceUserPool.get("bobby").put("cpu", 100.0);
         resourceUserPool.get("bobby").put("memory", 1000.0);
 
-
         resourceUserPool.put("derek", new HashMap<String, Number>());
         resourceUserPool.get("derek").put("cpu", 25.0);
         resourceUserPool.get("derek").put("memory", 250.0);
@@ -951,7 +944,6 @@ public class TestResourceAwareScheduler {
         Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
         Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
 
-
         for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
             Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }
@@ -1090,7 +1082,6 @@ public class TestResourceAwareScheduler {
         topoMap.put(topo5.getId(), topo5);
         topoMap.put(topo6.getId(), topo6);
 
-
         Topologies topologies = new Topologies(topoMap);
 
         ResourceAwareScheduler rs = new ResourceAwareScheduler();
@@ -1106,7 +1097,6 @@ public class TestResourceAwareScheduler {
         Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
         Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
 
-
         for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
             Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/a7f69135/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
index 2098f0c..dcd487f 100644
--- a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
+++ b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
@@ -185,7 +185,6 @@ public class TestUtilsForResourceAwareScheduler {
         }
 
         public void close() {
-
         }
 
         public void nextTuple() {
@@ -197,11 +196,9 @@ public class TestUtilsForResourceAwareScheduler {
         }
 
         public void ack(Object msgId) {
-
         }
 
         public void fail(Object msgId) {
-
         }
 
         public void declareOutputFields(OutputFieldsDeclarer declarer) {


[23/23] storm git commit: Merge branch 'STORM-898-merge'

Posted by da...@apache.org.
Merge branch 'STORM-898-merge'


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

Branch: refs/heads/master
Commit: c7164ccd4c8f8f9b3906e1f80d3c0ca7876099dd
Parents: 8f9ed06 eaf5b37
Author: Derek Dagit <de...@yahoo-inc.com>
Authored: Mon Dec 21 08:28:38 2015 -0600
Committer: Derek Dagit <de...@yahoo-inc.com>
Committed: Mon Dec 21 08:28:38 2015 -0600

----------------------------------------------------------------------
 CHANGELOG.md                                    |    1 +
 conf/defaults.yaml                              |    6 +
 conf/user-resource-pools-example.yaml           |   26 +
 .../Resource_Aware_Scheduler_overview.md        |  227 ++++
 ...ware_scheduler_default_eviction_strategy.svg |    3 +
 .../starter/ResourceAwareExampleTopology.java   |    9 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   14 +-
 storm-core/src/jvm/backtype/storm/Config.java   |   59 +-
 .../jvm/backtype/storm/scheduler/Cluster.java   |   71 +
 .../storm/scheduler/SupervisorDetails.java      |    8 +-
 .../backtype/storm/scheduler/Topologies.java    |   14 +
 .../storm/scheduler/TopologyDetails.java        |  157 ++-
 .../storm/scheduler/resource/RAS_Node.java      |  253 ++--
 .../storm/scheduler/resource/RAS_Nodes.java     |  160 +++
 .../resource/ResourceAwareScheduler.java        |  421 ++++--
 .../storm/scheduler/resource/ResourceUtils.java |   51 +
 .../scheduler/resource/SchedulingResult.java    |  116 ++
 .../scheduler/resource/SchedulingStatus.java    |   40 +
 .../backtype/storm/scheduler/resource/User.java |  346 +++++
 .../resource/strategies/IStrategy.java          |   37 -
 .../strategies/ResourceAwareStrategy.java       |  479 -------
 .../eviction/DefaultEvictionStrategy.java       |  126 ++
 .../strategies/eviction/IEvictionStrategy.java  |   47 +
 .../DefaultSchedulingPriorityStrategy.java      |   81 ++
 .../priority/ISchedulingPriorityStrategy.java   |   41 +
 .../DefaultResourceAwareStrategy.java           |  488 +++++++
 .../strategies/scheduling/IStrategy.java        |   50 +
 .../storm/validation/ConfigValidation.java      |   60 +-
 .../validation/ConfigValidationAnnotations.java |    9 +
 .../scheduler/resource_aware_scheduler_test.clj |  193 ++-
 .../jvm/backtype/storm/TestConfigValidate.java  |   84 ++
 .../resource/TestResourceAwareScheduler.java    | 1227 ++++++++++++++++++
 .../storm/scheduler/resource/TestUser.java      |  111 ++
 .../TestUtilsForResourceAwareScheduler.java     |  288 ++++
 34 files changed, 4419 insertions(+), 884 deletions(-)
----------------------------------------------------------------------



[21/23] storm git commit: Merge branch 'STORM-898' of https://github.com/jerrypeng/storm into STORM-898-merge

Posted by da...@apache.org.
Merge branch 'STORM-898' of https://github.com/jerrypeng/storm into STORM-898-merge


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

Branch: refs/heads/master
Commit: d790eb5abb1952be57f16f9a49f84eff4b0476b5
Parents: 8f9ed06 dc6d0f7
Author: Derek Dagit <de...@yahoo-inc.com>
Authored: Mon Dec 21 08:28:05 2015 -0600
Committer: Derek Dagit <de...@yahoo-inc.com>
Committed: Mon Dec 21 08:28:05 2015 -0600

----------------------------------------------------------------------
 conf/defaults.yaml                              |    6 +
 conf/user-resource-pools-example.yaml           |   26 +
 .../Resource_Aware_Scheduler_overview.md        |  227 ++++
 ...ware_scheduler_default_eviction_strategy.svg |    3 +
 .../starter/ResourceAwareExampleTopology.java   |    9 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   14 +-
 storm-core/src/jvm/backtype/storm/Config.java   |   59 +-
 .../jvm/backtype/storm/scheduler/Cluster.java   |   71 +
 .../storm/scheduler/SupervisorDetails.java      |    8 +-
 .../backtype/storm/scheduler/Topologies.java    |   14 +
 .../storm/scheduler/TopologyDetails.java        |  157 ++-
 .../storm/scheduler/resource/RAS_Node.java      |  253 ++--
 .../storm/scheduler/resource/RAS_Nodes.java     |  160 +++
 .../resource/ResourceAwareScheduler.java        |  421 ++++--
 .../storm/scheduler/resource/ResourceUtils.java |   51 +
 .../scheduler/resource/SchedulingResult.java    |  116 ++
 .../scheduler/resource/SchedulingStatus.java    |   40 +
 .../backtype/storm/scheduler/resource/User.java |  346 +++++
 .../resource/strategies/IStrategy.java          |   37 -
 .../strategies/ResourceAwareStrategy.java       |  479 -------
 .../eviction/DefaultEvictionStrategy.java       |  126 ++
 .../strategies/eviction/IEvictionStrategy.java  |   47 +
 .../DefaultSchedulingPriorityStrategy.java      |   81 ++
 .../priority/ISchedulingPriorityStrategy.java   |   41 +
 .../DefaultResourceAwareStrategy.java           |  488 +++++++
 .../strategies/scheduling/IStrategy.java        |   50 +
 .../storm/validation/ConfigValidation.java      |   60 +-
 .../validation/ConfigValidationAnnotations.java |    9 +
 .../scheduler/resource_aware_scheduler_test.clj |  193 ++-
 .../jvm/backtype/storm/TestConfigValidate.java  |   84 ++
 .../resource/TestResourceAwareScheduler.java    | 1227 ++++++++++++++++++
 .../storm/scheduler/resource/TestUser.java      |  111 ++
 .../TestUtilsForResourceAwareScheduler.java     |  288 ++++
 33 files changed, 4418 insertions(+), 884 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/d790eb5a/conf/defaults.yaml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/d790eb5a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/d790eb5a/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------


[18/23] storm git commit: making edits based on comments and removing unneccessary null checks

Posted by da...@apache.org.
making edits based on comments and removing unneccessary null checks


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/068a8c21
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/068a8c21
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/068a8c21

Branch: refs/heads/master
Commit: 068a8c21e9e4f5e213fca9eab86398685036ac9e
Parents: 45f637f
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Wed Dec 16 16:06:47 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Wed Dec 16 16:06:47 2015 -0600

----------------------------------------------------------------------
 .../storm/scheduler/SupervisorDetails.java      |  8 ++-----
 .../storm/scheduler/TopologyDetails.java        | 22 +++++++-------------
 .../storm/scheduler/resource/RAS_Node.java      |  2 +-
 .../storm/scheduler/resource/RAS_Nodes.java     |  8 +++----
 4 files changed, 14 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/068a8c21/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
index 5943ecf..94afb62 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
@@ -120,17 +120,13 @@ public class SupervisorDetails {
 
     public Double getTotalMemory() {
         Double totalMemory = getTotalResource(Config.SUPERVISOR_MEMORY_CAPACITY_MB);
-        if (totalMemory == null) {
-            throw new IllegalStateException("default value for " + Config.SUPERVISOR_MEMORY_CAPACITY_MB + " is not set!");
-        }
+        assert totalMemory != null;
         return totalMemory;
     }
 
     public Double getTotalCPU() {
         Double totalCPU = getTotalResource(Config.SUPERVISOR_CPU_CAPACITY);
-        if (totalCPU == null) {
-            throw new IllegalStateException("default value for " + Config.SUPERVISOR_CPU_CAPACITY + " is not set!");
-        }
+        assert totalCPU != null;
         return totalCPU;
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/068a8c21/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
index 357bec7..479986d 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
@@ -418,17 +418,12 @@ public class TopologyDetails {
      */
     public void addDefaultResforExec(ExecutorDetails exec) {
         Double topologyComponentCpuPcorePercent = Utils.getDouble(this.topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT), null);
-        if (topologyComponentCpuPcorePercent == null) {
-            LOG.warn("default value for " + Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT + " needs to be set!");
-        }
         Double topologyComponentResourcesOffheapMemoryMb = Utils.getDouble(this.topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB), null);
-        if (topologyComponentResourcesOffheapMemoryMb == null) {
-            LOG.warn("default value for " + Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB + " needs to be set!");
-        }
         Double topologyComponentResourcesOnheapMemoryMb = Utils.getDouble(this.topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB), null);
-        if (topologyComponentResourcesOnheapMemoryMb == null) {
-            LOG.warn("default value for " + Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB + " needs to be set!");
-        }
+
+        assert topologyComponentCpuPcorePercent != null;
+        assert topologyComponentResourcesOffheapMemoryMb != null;
+        assert topologyComponentResourcesOnheapMemoryMb != null;
 
         Map<String, Double> defaultResourceList = new HashMap<>();
         defaultResourceList.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, topologyComponentCpuPcorePercent);
@@ -447,13 +442,10 @@ public class TopologyDetails {
      */
     private void initConfigs() {
         this.topologyWorkerMaxHeapSize = Utils.getDouble(this.topologyConf.get(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB), null);
-        if (this.topologyWorkerMaxHeapSize == null) {
-            LOG.warn("default value for " + Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB + " needs to be set!");
-        }
         this.topologyPriority = Utils.getInt(this.topologyConf.get(Config.TOPOLOGY_PRIORITY), null);
-        if (this.topologyPriority == null) {
-            LOG.warn("default value for " + Config.TOPOLOGY_PRIORITY + " needs to be set!");
-        }
+
+        assert this.topologyWorkerMaxHeapSize != null;
+        assert this.topologyPriority != null;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/storm/blob/068a8c21/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
index a38d9f9..16c60cf 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
@@ -196,7 +196,7 @@ public class RAS_Node {
      * @param ws the slot to free
      */
     public void free(WorkerSlot ws) {
-        LOG.info("freeing ws {} on node {}", ws, _hostname);
+        LOG.info("freeing WorkerSlot {} on node {}", ws, _hostname);
         if (_freeSlots.contains(ws)) return;
         for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
             Set<WorkerSlot> slots = entry.getValue();

http://git-wip-us.apache.org/repos/asf/storm/blob/068a8c21/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
index abba0b0..bdf0cca 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Nodes.java
@@ -53,9 +53,9 @@ public class RAS_Nodes {
             LOG.debug("resources_mem: {}, resources_CPU: {}", sup.getTotalMemory(), sup.getTotalCPU());
             nodeIdToNode.put(sup.getId(), new RAS_Node(id, sup.getAllPorts(), isAlive, sup, cluster, topologies));
         }
-        for (Map.Entry<String, SchedulerAssignment> entry : cluster.getAssignments().entrySet()) {
-            String topId = entry.getValue().getTopologyId();
-            for (WorkerSlot workerSlot : entry.getValue().getSlots()) {
+        for (SchedulerAssignment assignment : cluster.getAssignments().values()) {
+            String topId = assignment.getTopologyId();
+            for (WorkerSlot workerSlot : assignment.getSlots()) {
                 String id = workerSlot.getNodeId();
                 RAS_Node node = nodeIdToNode.get(id);
                 if (node == null) {
@@ -69,7 +69,7 @@ public class RAS_Nodes {
                     node.addOrphanedSlot(workerSlot);
                 }
                 if (node.assignInternal(workerSlot, topId, true)) {
-                    LOG.warn("Bad scheduling state, " + workerSlot + " assigned multiple workers, unassigning everything...");
+                    LOG.warn("Bad scheduling state, {} assigned multiple workers, unassigning everything...", workerSlot);
                     node.free(workerSlot);
                 }
             }


[14/23] storm git commit: adding new eviction state diagram

Posted by da...@apache.org.
adding new eviction state diagram


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

Branch: refs/heads/master
Commit: 9a542a6934426eb43f996ac7437e40013643bd43
Parents: e45ccfe
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Thu Dec 10 16:34:51 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Thu Dec 10 16:34:51 2015 -0600

----------------------------------------------------------------------
 docs/images/resource_aware_scheduler_default_eviction_strategy.svg | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/9a542a69/docs/images/resource_aware_scheduler_default_eviction_strategy.svg
----------------------------------------------------------------------
diff --git a/docs/images/resource_aware_scheduler_default_eviction_strategy.svg b/docs/images/resource_aware_scheduler_default_eviction_strategy.svg
index 8a09c85..2c531eb 100644
--- a/docs/images/resource_aware_scheduler_default_eviction_strategy.svg
+++ b/docs/images/resource_aware_scheduler_default_eviction_strategy.svg
@@ -1,3 +1,3 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
-<svg xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" width="2238px" height="1523px" version="1.1" style="background-color: rgb(255, 255, 255);"><defs/><g transform="translate(0.5,0.5)"><path d="M 988.5 162 L 1042 212 L 988.5 262 L 935 212 Z" fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(955,200)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="65" height="22" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 67px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Yes/No</div></div></foreignObject><text x="33" y="21" fill="#000000" text-anchor="middle" font-size="20px" fo
 nt-family="Helvetica">[Not supported by viewer]</text></switch></g><rect x="742" y="182" width="120" height="60" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(743,187)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="116" height="49" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 116px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Schedule t<span>oplogy </span><i>t<sub>i</sub></i></div></div></foreignObject><text x="58" y="35" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 935 212 L 868.37 212" fill="none" stroke="#000000" stroke
 -miterlimit="10" pointer-events="none"/><path d="M 863.12 212 L 870.12 208.5 L 868.37 212 L 870.12 215.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1286 260 L 1286 293 L 1286.41 320.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1286.48 325.88 L 1282.88 318.93 L 1286.41 320.63 L 1289.88 318.83 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1145" y="165" width="280" height="95" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1146,189)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="276" height="46" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 276px; white-space: normal; text-align: center;"><
 div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Determine if topologies should be evicted</div></div></foreignObject><text x="138" y="33" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1040.93 213 L 1094 213 L 1138.63 213" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1143.88 213 L 1136.88 216.5 L 1138.63 213 L 1136.88 209.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 988 113 L 988 156.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 988 161.88 L 984.5 154.88 L 988 156.63 L 991.5 154.88 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><ellipse cx="989" cy="57" rx="113.5" ry="55" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(876,30)"><switch><f
 oreignObject style="overflow:visible;" pointer-events="all" width="223" height="52" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 223px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><font style="font-size: 20px">Can topology <i>t<sub>i </sub></i>from user <i>u<sub>i </sub></i>be scheduled?</font></div></div></foreignObject><text x="112" y="36" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1285.5 622 L 1346 681 L 1285.5 740 L 1225 681 Z" fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(1259,669)"><switch><foreignObject style="overflo
 w:visible;" pointer-events="all" width="51" height="22" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 51px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">&gt;0 / 0</div></div></foreignObject><text x="26" y="21" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1286.5 532 L 1286 577 L 1286 616.12" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1286 621.37 L 1282.5 614.37 L 1286 616.12 L 1289.5 614.37 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><ellipse cx="1287" cy="430" rx="262.5" ry="102.5" fill="#ffffff" stroke="#0000
 00" pointer-events="none"/><g transform="translate(1025,376)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="521" height="106" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 521px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><span>How much resources does topology </span><i>t</i><sub style="font-style: italic">i  </sub>need outside of <span>user </span><i>u</i><sub style="font-style: italic">i</sub>'s resource guarantee:<br /><div><i>u</i><sub>i</sub><span>'s </span>allocated resource above guarantee + additional resources needed above guarantee to schedule <i>t</i><sub>i</sub></div></div></div></foreignObject><text x="261" y="63" fill="#00
 0000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1346 681 L 1422 681 L 1444.63 681" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1449.88 681 L 1442.88 684.5 L 1444.63 681 L 1442.88 677.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1764" y="1002" width="205" height="94" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1765,1002)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="201" height="92" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 201px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:in
 herit;text-decoration:inherit;"><p>Determine if <i>t<sub>j </sub></i>is killed can we schedule <i>t<sub>i </sub>.</i></p></div></div></foreignObject><text x="101" y="56" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><rect x="1451" y="622" width="261" height="126" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1453,636)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="257" height="97" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 257px; white-space: normal;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><div style="text-align: center"><span style="line-height: 1.2">Find user </span>
 <i style="line-height: 1.2">u<sub>j</sub></i><span style="line-height: 1.2">, the user that has the most resources above his or her guarantee</span></div><div style="text-align: center"><br /></div></div></div></foreignObject><text x="129" y="59" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1866.5 1162 L 1866.5 1102.37" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1866.5 1097.12 L 1870 1104.12 L 1866.5 1102.37 L 1863 1104.12 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1813.54 1211 L 1472 1211 L 1138.37 1211.49" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1133.12 1211.5 L 1140.11 1207.99 L 1138.37 1211.49 L 1140.12 1214.99 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1866.5 1162 L 1920 1211.5 L 1866.5 1261 L 1813 1211.5 Z"
  fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(1833,1200)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="65" height="22" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 67px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><div style="text-align: center">Yes/No</div></div></div></foreignObject><text x="33" y="21" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 2118 1149 L 2118 685 L 1718.37 685" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1713.12 685 L 1720.12 681.5 L 1
 718.37 685 L 1720.12 688.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1996" y="1149" width="241" height="125" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1997,1130)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="237" height="161" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 237px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><p>Recalculate how much resources users have above their guarantee taken into account that <i>t</i><sub style="font-style: italic">j</sub>'s is killed.</p></div></div></foreignObject><text x="119" y="91" fill="#000000" text-anchor="middle" fo
 nt-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1920 1211.5 L 1989.63 1211.5" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1994.88 1211.5 L 1987.88 1215 L 1989.63 1211.5 L 1987.88 1208 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 766 808 L 766 835 L 766 857.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 766 862.88 L 762.5 855.88 L 766 857.63 L 769.5 855.88 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="632" y="554" width="269" height="254" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(633,581)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="265" height="199" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-siz
 e: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 265px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><div style="text-align: center"><span style="line-height: 1.2">Find user </span><i style="line-height: 1.2">u</i><sub style="font-style: italic">j</sub><span style="line-height: 1.2">, the user who has the most  resources above his or her resource guarantee &gt; </span><i>u</i><sub>i</sub><span>'s </span><span>allocated</span><span> resource above guarantee + additional resources needed above guarantee to schedule </span><i>t</i><sub>i</sub><br /></div></div></div></foreignObject><text x="133" y="110" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1225 681 L 907.37 681" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-eve
 nts="none"/><path d="M 902.12 681 L 909.12 677.5 L 907.37 681 L 909.12 684.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1286" y="1412" width="256" height="110" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1287,1443)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="252" height="46" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 252px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Topology cannot be scheduled. </div></div></foreignObject><text x="126" y="33" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></sw
 itch></g><path d="M 1866.5 949 L 1866.5 995.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1866.5 1000.88 L 1863 993.88 L 1866.5 995.63 L 1870 993.88 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1751" y="815" width="231" height="134" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1752,831)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="227" height="100" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 227px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><span>Find a topology </span><i>t<sub>j </sub></i><span>with the lowest prio
 rity that has been scheduled and belongs to user </span><i>u<sub>j</sub></i><span>.</span></div></div></foreignObject><text x="114" y="60" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1582 817 L 1645 882 L 1582 947 L 1519 882 Z" fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(1548,858)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="66" height="46" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 66px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">None?<div>Yes/No</div></div></div></foreignObject><text x="33"
  y="33" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1582 748 L 1582 783 L 1582 810.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1582 815.88 L 1578.5 808.88 L 1582 810.63 L 1585.5 808.88 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1645 882 L 1744.63 882" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1749.88 882 L 1742.88 885.5 L 1744.63 882 L 1742.88 878.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1368 957 L 1368 1185 L 1414 1185 L 1414 1406.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1414 1411.88 L 1410.5 1404.88 L 1414 1406.63 L 1417.5 1404.88 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1272" y="807" width="193" height="150"
  fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1273,822)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="189" height="118" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 189px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><span>Total cluster resources (+/- room for fragmention) &lt; Total resource guarantees</span></div></div></foreignObject><text x="95" y="69" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 835 933 L 868 933 L 894.63 933" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 899.88
  933 L 892.88 936.5 L 894.63 933 L 892.88 929.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 765 863 L 835 933 L 765 1003 L 695 933 Z" fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(731,909)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="66" height="46" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 66px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">None?<div>No/Yes</div></div></div></foreignObject><text x="33" y="33" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d
 ="M 986 1010 L 986 1467 L 1279.63 1467" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1284.88 1467 L 1277.88 1470.5 L 1279.63 1467 L 1277.88 1463.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="901" y="857" width="170" height="153" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(902,896)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="166" height="73" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 166px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Wont be fair to allocate <span>user </span><i>u</i><sub style="font-style: italic">i
  </sub>more resources</div></div></foreignObject><text x="83" y="47" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 478 1202 L 478 1239 L 478 1268.11" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 478 1273.36 L 474.5 1266.36 L 478 1268.11 L 481.5 1266.36 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="372" y="1092" width="210" height="110" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(373,1100)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="206" height="92" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 206px; white-space: normal; text-align: center;"><div x
 mlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><p>Determine if <i>t<sub>j </sub></i>is killed can we schedule <i>t<sub>i </sub>.</i></p></div></div></foreignObject><text x="103" y="56" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 415.53 1333 L 362 1333 L 314.37 1333" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 309.12 1333 L 316.12 1329.5 L 314.37 1333 L 316.12 1336.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 538.42 1335 L 1134 1335 L 1134 531 L 802 531 L 802 249.37" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 802 244.12 L 805.5 251.12 L 802 249.37 L 798.5 251.12 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 477.5 1274 L 540 1333.5 L 477.5 1393 L 415 1333.5 Z
 " fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(444,1322)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="65" height="22" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 67px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><div style="text-align: center">No/Yes</div></div></div></foreignObject><text x="33" y="21" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 154 1263 L 154 681 L 625.63 681" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 630.88 681 L 623.88 684.5 L 625.63
  681 L 623.88 677.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="2" y="1262" width="305" height="142" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(3,1264)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="301" height="137" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 301px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><p>Recalculate how much resources users have above their guarantee taken into account that <i>t</i><sub style="font-style: italic">j</sub>'s is killed.</p></div></div></foreignObject><text x="151" y="79" fill="#000000" text-anchor="middle" font-size="20px
 " font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 603 933 L 688.63 933" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 693.88 933 L 686.88 936.5 L 688.63 933 L 686.88 929.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 478 1010 L 478 1051 L 478 1085.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 478 1090.88 L 474.5 1083.88 L 478 1085.63 L 481.5 1083.88 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="353" y="856" width="250" height="154" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(354,882)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="246" height="100" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: 
 Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 246px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><span>Find a topology </span><i>t<sub>j </sub></i><span>with the lowest priority that has been scheduled and belongs to user </span><i>u</i><sub style="font-style: italic">j </sub></div></div></foreignObject><text x="123" y="60" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1519 882 L 1471.37 882" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1466.12 882 L 1473.12 878.5 L 1471.37 882 L 1473.12 885.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/></g></svg>
\ No newline at end of file
+<svg xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" width="2263px" height="1523px" version="1.1" content="%3Cmxfile%20type%3D%22device%22%20userAgent%3D%22Mozilla%2F5.0%20(Macintosh%3B%20Intel%20Mac%20OS%20X%2010_10_4)%20AppleWebKit%2F537.36%20(KHTML%2C%20like%20Gecko)%20Chrome%2F47.0.2526.73%20Safari%2F537.36%22%20version%3D%225.2.6.8%22%20editor%3D%22www.draw.io%22%3E%3Cdiagram%3E7Vzrk6JIEv9rOu7ugxMIPro%2Fzqt3P9xtbNxsxN5%2BpBGFW7Q8xLZn%2FvrNrMosqrQQpkHQvjGMUIuiHplZmb984F3wcf3yUx5uk3%2BJRZzd%2Bd7i5S74dOf7k%2FF0Ch%2FY8lW1BN50rFpWebpQbUbDl%2FRbTI0ete7TRbyzOhZCZEW6tRsjsdnEUWG1hXkuDna3pcjsWbfhimcsG75EYXba%2Bnu6KBLVeu%2FPyvaf43SV8Mzj2YO68hRGf65ysd%2FQfHd%2BsJQvdXkd8lhyo8FnoGIuBAyD39YvH%2BMMKck0CoLFzJ9HD7NoNguXI0Wcx2ad9RbyeEPLbDNcoIZ7DrM90egPYJD%2F%2BItQF3bFVyZenoj10x448OGQpEX8ZRtGeOUAsgJtSbHO4NcYvi7FpiDeAz2CD7Urpj09x3kRk7DJJtrBT7FYx0X%2BFbrQVd%2BjdZMgjlnADiVXx95ctSUmRz3qGJIkrfTYJc3gC5GtEQlJeAwSfomSeLEHuoF43%2FmzDCb%2FsNuGG%2Fi%2Bwu9im4kVTDkL10i6zdMOP7gnTG9
 25uaUG8oh90%2Fchhf1zWWz0apvP%2BHr4Nxk5jEzAwcz1cGymTnrnpe8FIOZJ%2FSKF6BT6KfIi0SsxCbMPpetH6SiiHEGJFgHtNyJfS7ZU3mAizBfxe4xSD5x2We5kcdZWKTPtqpsQ1s2Dr0S82ibLak7ZitVTV4lrT2Qs2JpBnU%2FxUWcr9MNKp4UTBMY160AVZOiSvd2idiDvQRjhtfj5zQqgKxXpw0m42PdTr8NdeDfO9TBA6GTLmlMAvuW1AHLTf%2F64LLqIH5Ji%2F9g87sp%2FfqDr2xgocYl%2FInXcIA%2BlAjbrnNKZBgdwiuzWKIgAxLG4s3sf3sEtPLCaCdJ9h46AG1eyosMMj5K%2FGLBG9JFlbCnFt9UgqXzeGeZizWsc7%2BL89qp991OLRXtjsAgcODR6KToWwHJgGngEqG4Dw%2FOJrY6Hk1P0Znvu6A2Y6ku5fXeJa%2BKO5KSMCW8gc6M86%2FTf5k82ESdzl2Ql4a1iUoUaIXLJjePy9hFOKNSlaz0r1N5aQ6darp1P4sDBg72USLXpPaKMG0h5EedumzsJdq%2Bn6HeKnW7akblnhZhlkan6h0GhF02UYubGDGmJ%2FbFDkI%2F%2BA1wqftGczvntHXdzrUSv9DOXdv8G7KMuSgDXWEebopYjabueMIdWYtZpM8tdtHI8zcphYtsRNIwy0QUSu%2FA3FT4JODYmJtDKYCbvXCxSIsUNcSRMCP75TCuewuIMWnz2BwWtCMItFpkv17L6z%2BQWiYj4c%2FJwTGMxNSnNtNI%2BF73ntBkCF%2B%2BHeomS3kOdVN0tztXSN76Ps9DvIs7bEW6KeA06JF%2FxYaS0bMHOwA289W6dAj3%2B%2FrDF7WCks96K69ivcsJVodpy%2BfoJPLQCNX%2F9%2FXQOkXtomMYXoTKyzuYeLvpIsiWOeZ9d07jQJve%2FTWGVANbeTy4YDvrCSuKQuCwS0
 REx8whQFIVO8wxbLMYgQVebZQ5joAyEhEcmWPbzDkGykAmR7w%2FHGn8Dic%2FGuYxhcQO%2B4iWNLiBxqsncjmYeAoax%2B472e6dDyz0igTPCO25SELUMAmwna%2BsxQ6bTHPOVjyRx0%2FgjYm83TDrZ2l3bH27lYJThFVj7WvOKU39KYuXeHsi8vQbHF1AOX2d4%2FmYzs4ZR9GfuRxFzmm2Ocfk1d%2BwnzihLZwBAcq89eAncrqq43CnEew04pt4Scc%2By2tD0Xwq2%2Fp3xnktFzA9ZZ68kappHH1ipUPz9oMW7ukenXNpmlB%2FIP3UKh7V6bkYWPqn9RkYhYf69jLGwT2ZBHYbArXUC7kNTIhzbsO%2FY6jNifawbYkqqkJiiFHwMwkl%2BmAUAjAlPQIfQPY%2FY4RGsHWMLoRRBIKCVCCA02%2BcrBraWW6P6cyA0%2BG9XwAa7SgcAkuocG8kgbJUgrx%2FKkgYli0Yp7Pzx7L%2FVhRwNlKIEKHsyTV34RENofT8Gbk6rPS8OSkzC1zRobHA1YyAUIeWapCg%2FKWxFeuA%2FnPNTL0bxq5TkrxzufxA2s7%2BcRWv7UZc%2Bu9NJLT37%2Fu1JhcKChwSRLcW7RwBguMujQIGzlyJMZbTTnaXAbpc7sR1s5FPaXxPFVWHysO0zzc2ovmlQjg9AgsoIrXj81OPbI%2BFKwhrWIkbBgFdampXycStJ27Y%2FPSPK44Xchra%2Fk0n7jErsQEdSxWXuhAI8wpHZ%2BzqhXrKoqmLtDmcZQo186XzOqCxC2%2FcFpzjKO01hCIrllZTskKQCp3EzqpTuskGulZ7SEEsGapk4hBLsLLNU4jiF0hVK%2BXxFMuwQVmsB%2FuUu32KM7FZaf%2B3k8KULpI%2FpylKe%2B7r97%2B9o9zGfOYo9A5cuY2gezPJHr1xBH4RkM4uazYNEPKGQr%2Fze%2FLiNRNOi0x
 0LslmwgWepHKFDK%2B73J5E%2BNzjN7SHwbEKR7Juh7pMunPUHe5pBlJCb%2F3phXmDMF%2FbhMYrJZqXVoNbfhOYsgc4nu13MuRkucZ%2Fl67z4wibhXxUYCnjFMs8XK0x1i02%2FzAjE6TxPR7VFcnAYMGNmuXpxHZe5%2FzQt2kQuHzdMghcYdSKwbeP8ykmWxO3HSgfrhfXCOcgwnmUYOfmcc5IUVyL9T0%2FBmeKNYu%2BJdbc2CrD3W3ph7YIw6S4SXovaxJelePWATSOUsB%2FVlwyx60pYRyn39WDczLkswxlglqloikIfOuPP1Rkk9cixx0blvX6PVCOwbJO4HpYUydwwZ1t6sj%2BtcpQDhGY7VgR8LOB50zdZKAUpV7cj7r126lbH%2Bmwsi7KYPtrRoU4pNz986Ys9G%2BiFo3OXs35lJC7j%2FrXbh%2F3r3CL%2B9N9DtoOVeaqF3OBeow35QSMbS9grOuErHCn47GYMQerW5n8brWLM2SkfYIhHQSlUxqkcIc4KtWlSz%2FqUnutS%2F0eDGW6PEeFq1CmJEmb4L8B1tWwKmqvpVdp%2Fw9S3XNLt1TbOppzdkYX9HOex1B0gRNGcZV4K0XnMke35tvUl6H0WX75%2F0FQ5bMNYRWc%2FyzzowTiUiUQl7Jqjah3%2Ffo74EcRdGiKAptWtZUzNNV9bYTO7LypzDEnD%2FvPHHOZw2WoV7oFwb3lGLzzIGWtfv8a5yksG6VZDteJvlfOb%2F1fkfadFJjw2dFYiOpZqv5f4%2FiGe%2Fq3yar%2BAf8b5Sv7T2r6cwja3b11ToP5Zpzuz%2Bq5Lhs7p2gdStjdLnR4olQrVGS9EFYq0YCLc0q%2BO%2BL7DJRNJcoE%2F45DDj%2FLP%2FxWTCj%2FQz34%2FBc%3D%3C%2Fdiagram%3E%3C%2Fmxfile%3E" style="background-color: rgb(25
 5, 255, 255);"><defs/><g transform="translate(0.5,0.5)"><path d="M 990.5 164 L 1044 214 L 990.5 264 L 937 214 Z" fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(956,201)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="65" height="22" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 67px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Yes/No</div></div></foreignObject><text x="33" y="21" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><rect x="744" y="184" width="120" height="60" fill="#ffffff" stroke="#000000" pointer-events
 ="none"/><g transform="translate(745,188)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="114" height="49" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 114px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Schedule t<span>oplogy </span><i>t<sub>i</sub></i></div></div></foreignObject><text x="57" y="35" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 937 214 L 870.37 214" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 865.12 214 L 872.12 210.5 L 870.37 214 L 872.12 217.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="
 10" pointer-events="none"/><path d="M 1286.14 262 L 1286.14 295.86 L 1289.71 295.86 L 1288.73 322.64" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1288.54 327.88 L 1285.3 320.76 L 1288.73 322.64 L 1292.29 321.02 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1147" y="167" width="280" height="95" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1148,190)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="274" height="46" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 274px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Determine if topolog
 ies should be evicted</div></div></foreignObject><text x="137" y="33" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1043.69 213.71 L 1096.86 213.71 L 1140.63 213.71" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1145.88 213.71 L 1138.88 217.21 L 1140.63 213.71 L 1138.88 210.21 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 989.71 113.71 L 989.71 157.35" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 989.71 162.6 L 986.21 155.6 L 989.71 157.35 L 993.21 155.6 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><ellipse cx="991" cy="59" rx="113.5" ry="55" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(878,31)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="221" height="52" requiredFeatures
 ="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 221px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><font style="font-size: 20px">Can topology <i>t<sub>i </sub></i>from user <i>u<sub>i </sub></i>be scheduled?</font></div></div></foreignObject><text x="111" y="36" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1287.5 624 L 1348 683 L 1287.5 742 L 1227 683 Z" fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(1260,670)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="51" height="22" requiredFeatures="http://www.w3.org/TR/SVG11
 /feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 51px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">&gt;0 / 0</div></div></foreignObject><text x="26" y="21" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1288.5 534 L 1289.71 578 L 1286.14 578 L 1286.14 618.96" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1286.14 624.21 L 1282.64 617.21 L 1286.14 618.96 L 1289.64 617.21 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><ellipse cx="1289" cy="432" rx="262.5" ry="102.5" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1027,377)"><switch><f
 oreignObject style="overflow:visible;" pointer-events="all" width="519" height="106" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 519px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><span>How much resources does topology </span><i>t</i><sub style="font-style: italic">i  </sub>need outside of <span>user </span><i>u</i><sub style="font-style: italic">i</sub>'s resource guarantee:<br /><div><i>u</i><sub>i</sub><span>'s </span>allocated resource above guarantee + additional resources needed above guarantee to schedule <i>t</i><sub>i</sub></div></div></div></foreignObject><text x="260" y="63" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not
  supported by viewer]</text></switch></g><path d="M 1346.54 681.57 L 1425.43 681.57 L 1446.63 681.57" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1451.88 681.57 L 1444.88 685.07 L 1446.63 681.57 L 1444.88 678.07 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1766" y="1004" width="205" height="94" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1767,991)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="199" height="116" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 199px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><p>Determine if <
 i>t<sub>j </sub></i>is evicted can we schedule <i>t<sub>i </sub>.</i></p></div></div></foreignObject><text x="100" y="68" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><rect x="1453" y="624" width="261" height="126" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1455,637)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="255" height="97" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 255px; white-space: normal;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><div style="text-align: center"><span style="line-height: 1.2">Find user </span><i style="line-height: 1.2">u<sub>j</sub></i><sp
 an style="line-height: 1.2">, the user that has the most resources above his or her guarantee</span></div><div style="text-align: center"><br /></div></div></div></foreignObject><text x="128" y="59" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1868.29 1164.2 L 1868.29 1131.57 L 1868.29 1104.37" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1868.29 1099.12 L 1871.79 1106.12 L 1868.29 1104.37 L 1864.79 1106.12 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1814.71 1213.71 L 1210.37 1213.71" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1205.12 1213.71 L 1212.12 1210.21 L 1210.37 1213.71 L 1212.12 1217.21 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1868.5 1164 L 1922 1213.5 L 1868.5 1263 L 1815 1213.5 Z" fill="#ffffff" stroke="#0
 00000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(1834,1201)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="65" height="22" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 67px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><div style="text-align: center">Yes/No</div></div></div></foreignObject><text x="33" y="21" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 2118.29 1133 L 2118.29 688.71 L 1721.08 688.71" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1715.83 688.71 L 1722.83 685.21 L 1721.08 688
 .71 L 1722.83 692.21 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1998" y="1133" width="266" height="162" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1999,1118)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="260" height="188" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 260px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><p>Recalculate how much resources users have above their guarantee taken into account that <i>t</i><sub style="font-style: italic">j</sub> is evicted. Add <i>t</i><sub>j </sub>to list L, a list of topologies to potentially evict</p></div></div></
 foreignObject><text x="130" y="104" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1921.77 1213.71 L 1961.14 1213.71 L 1991.63 1213.71" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1996.88 1213.71 L 1989.88 1217.21 L 1991.63 1213.71 L 1989.88 1210.21 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 768.29 810 L 768.29 838.71 L 768.29 859.92" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 768.29 865.17 L 764.79 858.17 L 768.29 859.92 L 771.79 858.17 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="634" y="556" width="269" height="254" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(635,582)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="263" height="199" requiredFeatures="http:
 //www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 263px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><div style="text-align: center"><span style="line-height: 1.2">Find user </span><i style="line-height: 1.2">u</i><sub style="font-style: italic">j</sub><span style="line-height: 1.2">, the user who has the most  resources above his or her resource guarantee &gt; </span><i>u</i><sub>i</sub><span>'s </span><span>allocated</span><span> resource above guarantee + additional resources needed above guarantee to schedule </span><i>t</i><sub>i</sub><br /></div></div></div></foreignObject><text x="132" y="110" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by
  viewer]</text></switch></g><path d="M 1228.46 681.57 L 1064.71 681.57 L 909.37 681.57" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 904.12 681.57 L 911.12 678.07 L 909.37 681.57 L 911.12 685.07 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1288" y="1414" width="256" height="110" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1289,1444)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="250" height="46" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 250px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Topology cannot be scheduled. </d
 iv></div></foreignObject><text x="125" y="33" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1868.29 951 L 1868.29 978 L 1868.29 997.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1868.29 1002.88 L 1864.79 995.88 L 1868.29 997.63 L 1871.79 995.88 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1753" y="817" width="231" height="134" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1754,832)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="225" height="100" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 225px; white-space: normal; text-align: center;"><div xmlns
 ="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><span>Find a topology </span><i>t<sub>j </sub></i><span>with the lowest priority that has been scheduled and belongs to user </span><i>u<sub>j</sub></i><span>.</span></div></div></foreignObject><text x="113" y="60" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1584 819 L 1647 884 L 1584 949 L 1521 884 Z" fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(1549,859)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="66" height="46" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 66px; white-space: nowrap; text-align: cen
 ter;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">None?<div>Yes/No</div></div></div></foreignObject><text x="33" y="33" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1582.57 750 L 1582.57 785.14 L 1582.57 814.11" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1582.57 819.36 L 1579.07 812.36 L 1582.57 814.11 L 1586.07 812.36 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1645.89 885.14 L 1700.43 885.14 L 1746.63 885.14" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1751.88 885.14 L 1744.88 888.64 L 1746.63 885.14 L 1744.88 881.64 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1371.86 960.14 L 1371.86 1185.14 L 1414.71 1185.14 L 1414.71 1407.35" fill="none" stroke
 ="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1414.71 1412.6 L 1411.21 1405.6 L 1414.71 1407.35 L 1418.21 1405.6 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1274" y="809" width="193" height="150" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1275,823)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="187" height="118" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 187px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><span>Total cluster resources (+/- room for fragmention) &lt; Total resource guarantees</span></div></div></foreignObject><text x="94" y
 ="69" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 836.86 935.14 L 868.29 935.14 L 896.63 935.14" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 901.88 935.14 L 894.88 938.64 L 896.63 935.14 L 894.88 931.64 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 767 865 L 837 935 L 767 1005 L 697 935 Z" fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(732,910)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="66" height="46" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 66px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.or
 g/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">None?<div>No/Yes</div></div></div></foreignObject><text x="33" y="33" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 989.71 1012 L 989.71 1467.29 L 1283.35 1467.29" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1288.6 1467.29 L 1281.6 1470.79 L 1283.35 1467.29 L 1281.6 1463.79 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="903" y="859" width="170" height="153" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(904,897)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="164" height="73" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 
 0, 0); line-height: 1.2; vertical-align: top; width: 164px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Wont be fair to allocate <span>user </span><i>u</i><sub style="font-style: italic">i </sub>more resources</div></div></foreignObject><text x="82" y="47" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 479 1204 L 479 1238.71 L 479 1270.11" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 479 1275.36 L 475.5 1268.36 L 479 1270.11 L 482.5 1268.36 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="374" y="1094" width="210" height="110" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(375,1089)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="204" height="1
 16" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 204px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><p>Determine if <i>t<sub>j </sub></i>is evicted can we schedule <i>t<sub>i </sub>.</i></p></div></div></foreignObject><text x="102" y="68" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 417.38 1335.14 L 364.71 1335.14 L 321.08 1335.14" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 315.83 1335.14 L 322.83 1331.64 L 321.08 1335.14 L 322.83 1338.64 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 541.62 1335.14
  L 1139.71 1335.14 L 1139.71 1255.8" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1139.71 1250.55 L 1143.21 1257.55 L 1139.71 1255.8 L 1136.21 1257.55 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 479.5 1276 L 542 1335.5 L 479.5 1395 L 417 1335.5 Z" fill="#ffffff" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><g transform="translate(445,1323)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="65" height="22" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 67px; white-space: nowrap; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><div style="text-align: center">No/Yes</div
 ></div></div></foreignObject><text x="33" y="21" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 157.57 1242.29 L 157.57 681.57 L 626.2 681.57" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 631.45 681.57 L 624.45 685.07 L 626.2 681.57 L 624.45 678.07 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="4" y="1241" width="310" height="186" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(5,1238)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="304" height="188" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 304px; white-space: normal; text-align: center;"><div xmlns="ht
 tp://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><p>Recalculate how much resources users have above their guarantee taken into account that <i>t</i><sub style="font-style: italic">j</sub> is evicted. Add <i>t</i><sub>j </sub><span>to list L when hold a list of topologies that may be evicted</span></p></div></div></foreignObject><text x="152" y="104" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 605 935.14 L 650.43 935.14 L 690.77 935.14" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 696.02 935.14 L 689.02 938.64 L 690.77 935.14 L 689.02 931.64 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 479 1012 L 479 1053 L 479 1087.63" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 479 1092.88 L 475.5 1085.88 L 479 1087.63 L 482.5 10
 85.88 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="355" y="858" width="250" height="154" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(356,883)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="244" height="100" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 244px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;"><span>Find a topology </span><i>t<sub>j </sub></i><span>with the lowest priority that has been scheduled and belongs to user </span><i>u</i><sub style="font-style: italic">j </sub></div></div></foreignObject><text x="122" y="60" fill="#000000" text-anchor="middle
 " font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g><path d="M 1521 884 L 1473.37 884" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1468.12 884 L 1475.12 880.5 L 1473.37 884 L 1475.12 887.5 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 1121.86 1181.57 L 1139.71 1181.57 L 1139.71 935.14 L 1114.71 935.14 L 1114.71 535.14 L 804 535.14 L 804 250.37" fill="none" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><path d="M 804 245.12 L 807.5 252.12 L 804 250.37 L 800.5 252.12 Z" fill="#000000" stroke="#000000" stroke-miterlimit="10" pointer-events="none"/><rect x="1073" y="1179" width="130" height="70" fill="#ffffff" stroke="#000000" pointer-events="none"/><g transform="translate(1074,1177)"><switch><foreignObject style="overflow:visible;" pointer-events="all" width="124" height="70" requiredFeatures="http://www.w3.org/TR/SVG11/feature#Extensibility"><div
  xmlns="http://www.w3.org/1999/xhtml" style="display: inline-block; font-size: 20px; font-family: Helvetica; color: rgb(0, 0, 0); line-height: 1.2; vertical-align: top; width: 124px; white-space: normal; text-align: center;"><div xmlns="http://www.w3.org/1999/xhtml" style="display:inline-block;text-align:inherit;text-decoration:inherit;">Evict topologies in list L</div></div></foreignObject><text x="62" y="45" fill="#000000" text-anchor="middle" font-size="20px" font-family="Helvetica">[Not supported by viewer]</text></switch></g></g></svg>
\ No newline at end of file


[05/23] storm git commit: adding checkpointing

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/9d3c864c/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
index 73a8c73..b41b039 100644
--- a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
+++ b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestResourceAwareScheduler.java
@@ -138,7 +138,6 @@ public class TestResourceAwareScheduler {
         topoMap.put(topo6.getId(), topo6);
 
         topologies = new Topologies(topoMap);
-        rs = new ResourceAwareScheduler();
         rs.prepare(config);
         rs.schedule(topologies, cluster);
 
@@ -293,7 +292,7 @@ public class TestResourceAwareScheduler {
 
         int fullyScheduled = 0;
         for (TopologyDetails topo : topoMap.values()) {
-            if(cluster.getStatusMap().get(topo.getId()).equals("Fully Scheduled")) {
+            if (TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId()))) {
                 fullyScheduled++;
             }
         }
@@ -302,4 +301,401 @@ public class TestResourceAwareScheduler {
         Assert.assertEquals("# of topologies running", 1, rs.getUser("jerry").getTopologiesRunning().size());
         Assert.assertEquals("# of topologies schedule pending", 0, rs.getUser("jerry").getTopologiesPending().size());
     }
+
+    /**
+     * The resources in the cluster is limited. In the first round of scheduling, all resources in the cluster is used.
+     * User jerry submits another toplogy.  Since user jerry has has his resource guarantees satisfied, and user bobby
+     * has exceeded his resource guarantee, topo-3 from user bobby should be eviced.
+     */
+    @Test
+    public void testEviction() {
+        INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
+        Map<String, Number> resourceMap = new HashMap<String, Number>();
+        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 100.0);
+        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1000.0);
+        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(4, 4, resourceMap);
+        Config config = new Config();
+        config.putAll(Utils.readDefaultConfig());
+        config.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 100.0);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 500);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 500);
+        Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
+        resourceUserPool.put("jerry", new HashMap<String, Number>());
+        resourceUserPool.get("jerry").put("cpu", 200.0);
+        resourceUserPool.get("jerry").put("memory", 2000.0);
+
+        resourceUserPool.put("bobby", new HashMap<String, Number>());
+        resourceUserPool.get("bobby").put("cpu", 100.0);
+        resourceUserPool.get("bobby").put("memory", 1000.0);
+
+        resourceUserPool.put("derek", new HashMap<String, Number>());
+        resourceUserPool.get("derek").put("cpu", 200.0);
+        resourceUserPool.get("derek").put("memory", 2000.0);
+
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
+        Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), config);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
+
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
+
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
+
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
+
+        Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
+        topoMap.put(topo1.getId(), topo1);
+        topoMap.put(topo2.getId(), topo2);
+        topoMap.put(topo3.getId(), topo3);
+        topoMap.put(topo4.getId(), topo4);
+
+        Topologies topologies = new Topologies(topoMap);
+
+        ResourceAwareScheduler rs = new ResourceAwareScheduler();
+
+        rs.prepare(config);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("bobby").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("bobby").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+
+        //user jerry submits another topology
+        topoMap.put(topo6.getId(), topo6);
+        topologies = new Topologies(topoMap);
+
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("bobby").getTopologiesRunning().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesAttempted()) {
+            Assert.assertFalse("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of attempted topologies", 1, rs.getUser("bobby").getTopologiesAttempted().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+    }
+
+    @Test
+    public void TestEvictMultipleTopologies() {
+        INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
+        Map<String, Number> resourceMap = new HashMap<String, Number>();
+        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 100.0);
+        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1000.0);
+        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(4, 4, resourceMap);
+        Config config = new Config();
+        config.putAll(Utils.readDefaultConfig());
+        config.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 100.0);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 500);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 500);
+        Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
+        resourceUserPool.put("jerry", new HashMap<String, Number>());
+        resourceUserPool.get("jerry").put("cpu", 200.0);
+        resourceUserPool.get("jerry").put("memory", 2000.0);
+
+        resourceUserPool.put("derek", new HashMap<String, Number>());
+        resourceUserPool.get("derek").put("cpu", 100.0);
+        resourceUserPool.get("derek").put("memory", 1000.0);
+
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
+        Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), config);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
+
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 2, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
+
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
+
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
+
+
+        Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
+        topoMap.put(topo2.getId(), topo2);
+        topoMap.put(topo3.getId(), topo3);
+        topoMap.put(topo4.getId(), topo4);
+        topoMap.put(topo5.getId(), topo5);
+
+        Topologies topologies = new Topologies(topoMap);
+
+        ResourceAwareScheduler rs = new ResourceAwareScheduler();
+
+        rs.prepare(config);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("bobby").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("bobby").getTopologiesAttempted().size());
+
+        //user jerry submits another topology
+        topoMap.put(topo1.getId(), topo1);
+        topologies = new Topologies(topoMap);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesAttempted()) {
+            Assert.assertFalse("correct topology to evict", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of attempted topologies", 2, rs.getUser("bobby").getTopologiesAttempted().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+        Assert.assertEquals("# of running topologies", 0, rs.getUser("bobby").getTopologiesRunning().size());
+    }
+
+    /**
+     * Eviction order:
+     * topo-3: since user bobby don't have any resource guarantees and topo-3 is the lowest priority for user bobby
+     * topo-2: since user bobby don't have any resource guarantees and topo-2 is the next lowest priority for user bobby
+     * topo-5: since user derek has exceeded his resource guarantee while user jerry has not.  topo-5 and topo-4 has the same priority
+     * but topo-4 was submitted earlier thus we choose that one to evict
+     */
+    @Test
+    public void TestEvictMultipleTopologiesFromMultipleUsersInCorrectOrder() {
+        INimbus iNimbus = new TestUtilsForResourceAwareScheduler.INimbusTest();
+        Map<String, Number> resourceMap = new HashMap<String, Number>();
+        resourceMap.put(Config.SUPERVISOR_CPU_CAPACITY, 100.0);
+        resourceMap.put(Config.SUPERVISOR_MEMORY_CAPACITY_MB, 1000.0);
+        Map<String, SupervisorDetails> supMap = TestUtilsForResourceAwareScheduler.genSupervisors(4, 4, resourceMap);
+        Config config = new Config();
+        config.putAll(Utils.readDefaultConfig());
+        config.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 100.0);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 500);
+        config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 500);
+        Map<String, Map<String, Number>> resourceUserPool = new HashMap<String, Map<String, Number>>();
+        resourceUserPool.put("jerry", new HashMap<String, Number>());
+        resourceUserPool.get("jerry").put("cpu", 300.0);
+        resourceUserPool.get("jerry").put("memory", 3000.0);
+
+        resourceUserPool.put("derek", new HashMap<String, Number>());
+        resourceUserPool.get("derek").put("cpu", 100.0);
+        resourceUserPool.get("derek").put("memory", 1000.0);
+
+        config.put(Config.RESOURCE_AWARE_SCHEDULER_USER_POOLS, resourceUserPool);
+        Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), config);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "jerry");
+
+        TopologyDetails topo1 = TestUtilsForResourceAwareScheduler.getTopology("topo-1", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo6 = TestUtilsForResourceAwareScheduler.getTopology("topo-6", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo7 = TestUtilsForResourceAwareScheduler.getTopology("topo-7", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "bobby");
+
+        TopologyDetails topo2 = TestUtilsForResourceAwareScheduler.getTopology("topo-2", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 10);
+        TopologyDetails topo3 = TestUtilsForResourceAwareScheduler.getTopology("topo-3", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 20);
+
+        config.put(Config.TOPOLOGY_SUBMITTER_USER, "derek");
+
+        TopologyDetails topo4 = TestUtilsForResourceAwareScheduler.getTopology("topo-4", config, 1, 0, 1, 0, Time.currentTimeSecs() - 2, 30);
+        TopologyDetails topo5 = TestUtilsForResourceAwareScheduler.getTopology("topo-5", config, 1, 0, 1, 0, Time.currentTimeSecs() - 15, 30);
+
+        Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
+        topoMap.put(topo2.getId(), topo2);
+        topoMap.put(topo3.getId(), topo3);
+        topoMap.put(topo4.getId(), topo4);
+        topoMap.put(topo5.getId(), topo5);
+
+        Topologies topologies = new Topologies(topoMap);
+
+        ResourceAwareScheduler rs = new ResourceAwareScheduler();
+
+        rs.prepare(config);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("bobby").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("bobby").getTopologiesAttempted().size());
+
+        //user jerry submits another topology
+        topoMap.put(topo1.getId(), topo1);
+        topologies = new Topologies(topoMap);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesAttempted()) {
+            Assert.assertFalse("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of attempted topologies", 1, rs.getUser("bobby").getTopologiesAttempted().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("bobby").getTopologiesRunning().size());
+        Assert.assertEquals("correct topology to evict", rs.getUser("bobby").getTopologiesAttempted().iterator().next().getName(), "topo-3");
+
+        topoMap.put(topo6.getId(), topo6);
+        topologies = new Topologies(topoMap);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 2, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesAttempted()) {
+            Assert.assertFalse(TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of attempted topologies", 2, rs.getUser("bobby").getTopologiesAttempted().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+        Assert.assertEquals("# of running topologies", 0, rs.getUser("bobby").getTopologiesRunning().size());
+
+        Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-2", rs.getUser("bobby").getTopologiesAttempted()) != null);
+        Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-3", rs.getUser("bobby").getTopologiesAttempted()) != null);
+
+        topoMap.put(topo7.getId(), topo7);
+        topologies = new Topologies(topoMap);
+        rs.schedule(topologies, cluster);
+
+        for (TopologyDetails topo : rs.getUser("jerry").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 3, rs.getUser("jerry").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("jerry").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 0, rs.getUser("jerry").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("jerry").getTopologiesInvalid().size());
+
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesRunning()) {
+            Assert.assertTrue("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        for (TopologyDetails topo : rs.getUser("derek").getTopologiesAttempted()) {
+            Assert.assertFalse("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of running topologies", 1, rs.getUser("derek").getTopologiesRunning().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("derek").getTopologiesPending().size());
+        Assert.assertEquals("# of attempted topologies", 1, rs.getUser("derek").getTopologiesAttempted().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("derek").getTopologiesInvalid().size());
+        Assert.assertEquals("correct topology to evict", rs.getUser("derek").getTopologiesAttempted().iterator().next().getName(), "topo-4");
+
+        for (TopologyDetails topo : rs.getUser("bobby").getTopologiesAttempted()) {
+            Assert.assertFalse("Assert scheduling topology success", TestUtilsForResourceAwareScheduler.assertStatusSuccess(cluster.getStatusMap().get(topo.getId())));
+        }
+        Assert.assertEquals("# of attempted topologies", 2, rs.getUser("bobby").getTopologiesAttempted().size());
+        Assert.assertEquals("# of pending topologies", 0, rs.getUser("bobby").getTopologiesPending().size());
+        Assert.assertEquals("# of invalid topologies", 0, rs.getUser("bobby").getTopologiesInvalid().size());
+        Assert.assertEquals("# of running topologies", 0, rs.getUser("bobby").getTopologiesRunning().size());
+
+        Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-2", rs.getUser("bobby").getTopologiesAttempted()) != null);
+        Assert.assertTrue("correct topology to evict", TestUtilsForResourceAwareScheduler.findTopologyInSetFromName("topo-3", rs.getUser("bobby").getTopologiesAttempted()) != null);
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3c864c/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
index d4177c3..7721300 100644
--- a/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
+++ b/storm-core/test/jvm/backtype/storm/scheduler/resource/TestUtilsForResourceAwareScheduler.java
@@ -22,6 +22,7 @@ import backtype.storm.Config;
 import backtype.storm.generated.Bolt;
 import backtype.storm.generated.SpoutSpec;
 import backtype.storm.generated.StormTopology;
+import backtype.storm.generated.TopologySummary;
 import backtype.storm.scheduler.ExecutorDetails;
 import backtype.storm.scheduler.INimbus;
 import backtype.storm.scheduler.IScheduler;
@@ -53,6 +54,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 public class TestUtilsForResourceAwareScheduler {
     private static final Logger LOG = LoggerFactory.getLogger(TestUtilsForResourceAwareScheduler.class);
@@ -131,6 +137,7 @@ public class TestUtilsForResourceAwareScheduler {
         conf.putAll(config);
         conf.put(Config.TOPOLOGY_PRIORITY, priority);
         conf.put(Config.TOPOLOGY_NAME, name);
+        conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE);
         StormTopology topology = buildTopology(numSpout,numBolt, spoutParallelism, boltParallelism);
         TopologyDetails topo = new TopologyDetails(name + "-" + launchTime, conf, topology,
                 0,
@@ -262,4 +269,25 @@ public class TestUtilsForResourceAwareScheduler {
             return null;
         }
     }
+
+    private static boolean isContain(String source, String subItem){
+        String pattern = "\\b"+subItem+"\\b";
+        Pattern p=Pattern.compile(pattern, Pattern.CASE_INSENSITIVE);
+        Matcher m=p.matcher(source);
+        return m.find();
+    }
+
+    public static boolean assertStatusSuccess(String status) {
+        return isContain(status, "fully") && isContain(status, "scheduled") && !isContain(status, "unsuccessful");
+    }
+
+    public static TopologyDetails findTopologyInSetFromName(String topoName, Set<TopologyDetails> set) {
+        TopologyDetails ret = null;
+        for(TopologyDetails entry : set) {
+            if(entry.getName().equals(topoName)) {
+                ret = entry;
+            }
+        }
+        return ret;
+    }
 }


[17/23] storm git commit: renaming RAS_Node.java variables

Posted by da...@apache.org.
renaming RAS_Node.java variables


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/45f637f9
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/45f637f9
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/45f637f9

Branch: refs/heads/master
Commit: 45f637f9ba902f821412982f48f6ac12eff00b40
Parents: 7676e0d
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Wed Dec 16 13:19:57 2015 -0600
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Wed Dec 16 13:19:57 2015 -0600

----------------------------------------------------------------------
 .../storm/scheduler/resource/RAS_Node.java      | 228 +++++++++----------
 1 file changed, 114 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/45f637f9/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
index be39d2a..a38d9f9 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
@@ -43,76 +43,76 @@ import backtype.storm.scheduler.WorkerSlot;
  */
 public class RAS_Node {
     private static final Logger LOG = LoggerFactory.getLogger(RAS_Node.class);
-    private Map<String, Set<WorkerSlot>> topIdToUsedSlots = new HashMap<String, Set<WorkerSlot>>();
-    private Set<WorkerSlot> freeSlots = new HashSet<WorkerSlot>();
-    private final String nodeId;
-    private String hostname;
-    private boolean isAlive;
-    private SupervisorDetails sup;
-    private Double availMemory;
-    private Double availCPU;
-    private Cluster cluster;
-    private Topologies topologies;
+    private Map<String, Set<WorkerSlot>> _topIdToUsedSlots = new HashMap<String, Set<WorkerSlot>>();
+    private Set<WorkerSlot> _freeSlots = new HashSet<WorkerSlot>();
+    private final String _nodeId;
+    private String _hostname;
+    private boolean _isAlive;
+    private SupervisorDetails _sup;
+    private Double _availMemory;
+    private Double _availCPU;
+    private Cluster _cluster;
+    private Topologies _topologies;
 
     public RAS_Node(String nodeId, Set<Integer> allPorts, boolean isAlive,
                     SupervisorDetails sup, Cluster cluster, Topologies topologies) {
-        this.nodeId = nodeId;
-        this.isAlive = isAlive;
-        if (this.isAlive && allPorts != null) {
+        _nodeId = nodeId;
+        _isAlive = isAlive;
+        if (_isAlive && allPorts != null) {
             for (int port : allPorts) {
-                this.freeSlots.add(new WorkerSlot(this.nodeId, port));
+                _freeSlots.add(new WorkerSlot(_nodeId, port));
             }
-            this.sup = sup;
-            this.hostname = sup.getHost();
-            this.availMemory = getTotalMemoryResources();
-            this.availCPU = getTotalCpuResources();
+            _sup = sup;
+            _hostname = sup.getHost();
+            _availMemory = getTotalMemoryResources();
+            _availCPU = getTotalCpuResources();
         }
-        this.cluster = cluster;
-        this.topologies = topologies;
+        _cluster = cluster;
+        _topologies = topologies;
     }
 
     public String getId() {
-        return this.nodeId;
+        return _nodeId;
     }
 
     public String getHostname() {
-        return this.hostname;
+        return _hostname;
     }
 
     public Collection<WorkerSlot> getFreeSlots() {
-        return this.freeSlots;
+        return _freeSlots;
     }
 
     public Collection<WorkerSlot> getUsedSlots() {
         Collection<WorkerSlot> ret = new LinkedList<WorkerSlot>();
-        for (Collection<WorkerSlot> workers : this.topIdToUsedSlots.values()) {
+        for (Collection<WorkerSlot> workers : _topIdToUsedSlots.values()) {
             ret.addAll(workers);
         }
         return ret;
     }
 
     public boolean isAlive() {
-        return this.isAlive;
+        return _isAlive;
     }
 
     /**
      * @return a collection of the topology ids currently running on this node
      */
     public Collection<String> getRunningTopologies() {
-        return this.topIdToUsedSlots.keySet();
+        return _topIdToUsedSlots.keySet();
     }
 
     public boolean isTotallyFree() {
-        return this.topIdToUsedSlots.isEmpty();
+        return _topIdToUsedSlots.isEmpty();
     }
 
     public int totalSlotsFree() {
-        return this.freeSlots.size();
+        return _freeSlots.size();
     }
 
     public int totalSlotsUsed() {
         int total = 0;
-        for (Set<WorkerSlot> slots : this.topIdToUsedSlots.values()) {
+        for (Set<WorkerSlot> slots : _topIdToUsedSlots.values()) {
             total += slots.size();
         }
         return total;
@@ -124,7 +124,7 @@ public class RAS_Node {
 
     public int totalSlotsUsed(String topId) {
         int total = 0;
-        Set<WorkerSlot> slots = this.topIdToUsedSlots.get(topId);
+        Set<WorkerSlot> slots = _topIdToUsedSlots.get(topId);
         if (slots != null) {
             total = slots.size();
         }
@@ -132,42 +132,42 @@ public class RAS_Node {
     }
 
     private void validateSlot(WorkerSlot ws) {
-        if (!this.nodeId.equals(ws.getNodeId())) {
+        if (!_nodeId.equals(ws.getNodeId())) {
             throw new IllegalArgumentException(
                     "Trying to add a slot to the wrong node " + ws +
-                            " is not a part of " + this.nodeId);
+                            " is not a part of " + _nodeId);
         }
     }
 
     void addOrphanedSlot(WorkerSlot ws) {
-        if (this.isAlive) {
+        if (_isAlive) {
             throw new IllegalArgumentException("Orphaned Slots " +
                     "only are allowed on dead nodes.");
         }
         validateSlot(ws);
-        if (this.freeSlots.contains(ws)) {
+        if (_freeSlots.contains(ws)) {
             return;
         }
-        for (Set<WorkerSlot> used : this.topIdToUsedSlots.values()) {
+        for (Set<WorkerSlot> used : _topIdToUsedSlots.values()) {
             if (used.contains(ws)) {
                 return;
             }
         }
-        this.freeSlots.add(ws);
+        _freeSlots.add(ws);
     }
 
     boolean assignInternal(WorkerSlot ws, String topId, boolean dontThrow) {
         validateSlot(ws);
-        if (!this.freeSlots.remove(ws)) {
+        if (!_freeSlots.remove(ws)) {
             if (dontThrow) {
                 return true;
             }
             throw new IllegalStateException("Assigning a slot that was not free " + ws);
         }
-        Set<WorkerSlot> usedSlots = this.topIdToUsedSlots.get(topId);
+        Set<WorkerSlot> usedSlots = _topIdToUsedSlots.get(topId);
         if (usedSlots == null) {
             usedSlots = new HashSet<WorkerSlot>();
-            this.topIdToUsedSlots.put(topId, usedSlots);
+            _topIdToUsedSlots.put(topId, usedSlots);
         }
         usedSlots.add(ws);
         return false;
@@ -177,18 +177,18 @@ public class RAS_Node {
      * Free all slots on this node.  This will update the Cluster too.
      */
     public void freeAllSlots() {
-        if (!this.isAlive) {
-            LOG.warn("Freeing all slots on a dead node {} ", this.nodeId);
-        }
-        for (Entry<String, Set<WorkerSlot>> entry : this.topIdToUsedSlots.entrySet()) {
-            this.cluster.freeSlots(entry.getValue());
-            this.availCPU = this.getTotalCpuResources();
-            this.availMemory = this.getAvailableMemoryResources();
-            if (this.isAlive) {
-                this.freeSlots.addAll(entry.getValue());
+        if (!_isAlive) {
+            LOG.warn("Freeing all slots on a dead node {} ", _nodeId);
+        }
+        for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
+            _cluster.freeSlots(entry.getValue());
+            _availCPU = getTotalCpuResources();
+            _availMemory = getAvailableMemoryResources();
+            if (_isAlive) {
+                _freeSlots.addAll(entry.getValue());
             }
         }
-        this.topIdToUsedSlots = new HashMap<String, Set<WorkerSlot>>();
+        _topIdToUsedSlots = new HashMap<String, Set<WorkerSlot>>();
     }
 
     /**
@@ -196,24 +196,24 @@ public class RAS_Node {
      * @param ws the slot to free
      */
     public void free(WorkerSlot ws) {
-        LOG.info("freeing ws {} on node {}", ws, this.hostname);
-        if (this.freeSlots.contains(ws)) return;
-        for (Entry<String, Set<WorkerSlot>> entry : this.topIdToUsedSlots.entrySet()) {
+        LOG.info("freeing ws {} on node {}", ws, _hostname);
+        if (_freeSlots.contains(ws)) return;
+        for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
             Set<WorkerSlot> slots = entry.getValue();
-            double memUsed = this.getMemoryUsedByWorker(ws);
-            double cpuUsed = this.getCpuUsedByWorker(ws);
+            double memUsed = getMemoryUsedByWorker(ws);
+            double cpuUsed = getCpuUsedByWorker(ws);
             if (slots.remove(ws)) {
-                this.cluster.freeSlot(ws);
-                if (this.isAlive) {
-                    this.freeSlots.add(ws);
+                _cluster.freeSlot(ws);
+                if (_isAlive) {
+                    _freeSlots.add(ws);
                 }
-                this.freeMemory(memUsed);
-                this.freeCPU(cpuUsed);
+                freeMemory(memUsed);
+                freeCPU(cpuUsed);
                 return;
             }
         }
         throw new IllegalArgumentException("Tried to free a slot that was not" +
-                " part of this node " + this.nodeId);
+                " part of this node " + _nodeId);
     }
 
     /**
@@ -221,45 +221,45 @@ public class RAS_Node {
      * @param topId the topology to free slots for
      */
     public void freeTopology(String topId) {
-        Set<WorkerSlot> slots = this.topIdToUsedSlots.get(topId);
+        Set<WorkerSlot> slots = _topIdToUsedSlots.get(topId);
         if (slots == null || slots.isEmpty()) {
             return;
         }
         for (WorkerSlot ws : slots) {
-            this.cluster.freeSlot(ws);
-            this.freeMemory(this.getMemoryUsedByWorker(ws));
-            this.freeCPU(this.getCpuUsedByWorker(ws));
-            if (this.isAlive) {
-                this.freeSlots.add(ws);
+            _cluster.freeSlot(ws);
+            freeMemory(getMemoryUsedByWorker(ws));
+            freeCPU(getCpuUsedByWorker(ws));
+            if (_isAlive) {
+                _freeSlots.add(ws);
             }
         }
-        this.topIdToUsedSlots.remove(topId);
+        _topIdToUsedSlots.remove(topId);
     }
 
     private void freeMemory(double amount) {
-        LOG.debug("freeing {} memory on node {}...avail mem: {}", amount, this.getHostname(), this.availMemory);
-        if((this.availMemory + amount) > this.getTotalCpuResources()) {
+        LOG.debug("freeing {} memory on node {}...avail mem: {}", amount, getHostname(), _availMemory);
+        if((_availMemory + amount) > getTotalCpuResources()) {
             LOG.warn("Freeing more memory than there exists!");
             return;
         }
-        this.availMemory += amount;
+        _availMemory += amount;
     }
 
     private void freeCPU(double amount) {
-        LOG.debug("freeing {} CPU on node...avail CPU: {}", amount, this.getHostname(), this.availCPU);
-        if ((this.availCPU + amount) > this.getAvailableCpuResources()) {
+        LOG.debug("freeing {} CPU on node...avail CPU: {}", amount, getHostname(), _availCPU);
+        if ((_availCPU + amount) > getAvailableCpuResources()) {
             LOG.warn("Freeing more CPU than there exists!");
             return;
         }
-        this.availCPU += amount;
+        _availCPU += amount;
     }
 
     public double getMemoryUsedByWorker(WorkerSlot ws) {
-        TopologyDetails topo = this.findTopologyUsingWorker(ws);
+        TopologyDetails topo = findTopologyUsingWorker(ws);
         if (topo == null) {
             return 0.0;
         }
-        Collection<ExecutorDetails> execs = this.getExecutors(ws, this.cluster);
+        Collection<ExecutorDetails> execs = getExecutors(ws, _cluster);
         double totalMemoryUsed = 0.0;
         for (ExecutorDetails exec : execs) {
             totalMemoryUsed += topo.getTotalMemReqTask(exec);
@@ -268,11 +268,11 @@ public class RAS_Node {
     }
 
     public double getCpuUsedByWorker(WorkerSlot ws) {
-        TopologyDetails topo = this.findTopologyUsingWorker(ws);
+        TopologyDetails topo = findTopologyUsingWorker(ws);
         if (topo == null) {
             return 0.0;
         }
-        Collection<ExecutorDetails> execs = this.getExecutors(ws, this.cluster);
+        Collection<ExecutorDetails> execs = getExecutors(ws, _cluster);
         double totalCpuUsed = 0.0;
         for (ExecutorDetails exec : execs) {
             totalCpuUsed += topo.getTotalCpuReqTask(exec);
@@ -281,12 +281,12 @@ public class RAS_Node {
     }
 
     public TopologyDetails findTopologyUsingWorker(WorkerSlot ws) {
-        for (Entry<String, Set<WorkerSlot>> entry : this.topIdToUsedSlots.entrySet()) {
+        for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
             String topoId = entry.getKey();
             Set<WorkerSlot> workers = entry.getValue();
             for (WorkerSlot worker : workers) {
                 if (worker.getNodeId().equals(ws.getNodeId()) && worker.getPort() == ws.getPort()) {
-                    return this.topologies.getById(topoId);
+                    return _topologies.getById(topoId);
                 }
             }
         }
@@ -321,24 +321,24 @@ public class RAS_Node {
     }
 
     public void assign(WorkerSlot target, TopologyDetails td, Collection<ExecutorDetails> executors) {
-        if (!this.isAlive) {
-            throw new IllegalStateException("Trying to adding to a dead node " + this.nodeId);
+        if (!_isAlive) {
+            throw new IllegalStateException("Trying to adding to a dead node " + _nodeId);
         }
-        if (this.freeSlots.isEmpty()) {
-            throw new IllegalStateException("Trying to assign to a full node " + this.nodeId);
+        if (_freeSlots.isEmpty()) {
+            throw new IllegalStateException("Trying to assign to a full node " + _nodeId);
         }
         if (executors.size() == 0) {
-            LOG.warn("Trying to assign nothing from " + td.getId() + " to " + this.nodeId + " (Ignored)");
+            LOG.warn("Trying to assign nothing from " + td.getId() + " to " + _nodeId + " (Ignored)");
         }
 
         if (target == null) {
-            target = this.freeSlots.iterator().next();
+            target = _freeSlots.iterator().next();
         }
-        if (!this.freeSlots.contains(target)) {
-            throw new IllegalStateException("Trying to assign already used slot" + target.getPort() + "on node " + this.nodeId);
+        if (!_freeSlots.contains(target)) {
+            throw new IllegalStateException("Trying to assign already used slot" + target.getPort() + "on node " + _nodeId);
         } else {
             allocateResourceToSlot(td, executors, target);
-            this.cluster.assign(target, td.getId(), executors);
+            _cluster.assign(target, td.getId(), executors);
             assignInternal(target, td.getId(), false);
         }
     }
@@ -350,27 +350,27 @@ public class RAS_Node {
      * @param executors the executors to run in that slot.
      */
     public void assign(TopologyDetails td, Collection<ExecutorDetails> executors) {
-        this.assign(null, td, executors);
+        assign(null, td, executors);
     }
 
     @Override
     public boolean equals(Object other) {
         if (other instanceof RAS_Node) {
-            return this.nodeId.equals(((RAS_Node) other).nodeId);
+            return _nodeId.equals(((RAS_Node) other)._nodeId);
         }
         return false;
     }
 
     @Override
     public int hashCode() {
-        return this.nodeId.hashCode();
+        return _nodeId.hashCode();
     }
 
     @Override
     public String toString() {
-        return "{Node: " + ((this.sup == null) ? "null (possibly down)" : this.sup.getHost())
-                + ", AvailMem: " + ((this.availMemory == null) ? "N/A" : this.availMemory.toString())
-                + ", this.availCPU: " + ((this.availCPU == null) ? "N/A" : this.availCPU.toString()) + "}";
+        return "{Node: " + ((_sup == null) ? "null (possibly down)" : _sup.getHost())
+                + ", AvailMem: " + ((_availMemory == null) ? "N/A" : _availMemory.toString())
+                + ", AvailCPU: " + ((_availCPU == null) ? "N/A" : _availCPU.toString()) + "}";
     }
 
     public static int countSlotsUsed(String topId, Collection<RAS_Node> nodes) {
@@ -433,7 +433,7 @@ public class RAS_Node {
      * @param amount the amount to set as available memory
      */
     public void setAvailableMemory(Double amount) {
-        this.availMemory = amount;
+        _availMemory = amount;
     }
 
     /**
@@ -441,10 +441,10 @@ public class RAS_Node {
      * @return the available memory for this node
      */
     public Double getAvailableMemoryResources() {
-        if (this.availMemory == null) {
+        if (_availMemory == null) {
             return 0.0;
         }
-        return this.availMemory;
+        return _availMemory;
     }
 
     /**
@@ -452,8 +452,8 @@ public class RAS_Node {
      * @return the total memory for this node
      */
     public Double getTotalMemoryResources() {
-        if (this.sup != null && this.sup.getTotalMemory() != null) {
-            return this.sup.getTotalMemory();
+        if (_sup != null && _sup.getTotalMemory() != null) {
+            return _sup.getTotalMemory();
         } else {
             return 0.0;
         }
@@ -465,12 +465,12 @@ public class RAS_Node {
      * @return the current available memory for this node after consumption
      */
     public Double consumeMemory(Double amount) {
-        if (amount > this.availMemory) {
-            LOG.error("Attempting to consume more memory than available! Needed: {}, we only have: {}", amount, this.availMemory);
+        if (amount > _availMemory) {
+            LOG.error("Attempting to consume more memory than available! Needed: {}, we only have: {}", amount, _availMemory);
             return null;
         }
-        this.availMemory = this.availMemory - amount;
-        return this.availMemory;
+        _availMemory = _availMemory - amount;
+        return _availMemory;
     }
 
     /**
@@ -478,10 +478,10 @@ public class RAS_Node {
      * @return the available cpu for this node
      */
     public Double getAvailableCpuResources() {
-        if (this.availCPU == null) {
+        if (_availCPU == null) {
             return 0.0;
         }
-        return this.availCPU;
+        return _availCPU;
     }
 
     /**
@@ -489,8 +489,8 @@ public class RAS_Node {
      * @return the total cpu for this node
      */
     public Double getTotalCpuResources() {
-        if (this.sup != null && this.sup.getTotalCPU() != null) {
-            return this.sup.getTotalCPU();
+        if (_sup != null && _sup.getTotalCPU() != null) {
+            return _sup.getTotalCPU();
         } else {
             return 0.0;
         }
@@ -502,12 +502,12 @@ public class RAS_Node {
      * @return the current available cpu for this node after consumption
      */
     public Double consumeCPU(Double amount) {
-        if (amount > this.availCPU) {
-            LOG.error("Attempting to consume more CPU than available! Needed: {}, we only have: {}", amount, this.availCPU);
+        if (amount > _availCPU) {
+            LOG.error("Attempting to consume more CPU than available! Needed: {}, we only have: {}", amount, _availCPU);
             return null;
         }
-        this.availCPU = this.availCPU - amount;
-        return this.availCPU;
+        _availCPU = _availCPU - amount;
+        return _availCPU;
     }
 
     /**
@@ -518,11 +518,11 @@ public class RAS_Node {
     public void consumeResourcesforTask(ExecutorDetails exec, TopologyDetails topo) {
         Double taskMemReq = topo.getTotalMemReqTask(exec);
         Double taskCpuReq = topo.getTotalCpuReqTask(exec);
-        this.consumeCPU(taskCpuReq);
-        this.consumeMemory(taskMemReq);
+        consumeCPU(taskCpuReq);
+        consumeMemory(taskMemReq);
     }
 
     public Map<String, Set<WorkerSlot>> getTopoIdTousedSlots() {
-        return this.topIdToUsedSlots;
+        return _topIdToUsedSlots;
     }
 }