From aee3e86d9b4bec99f16ac6635b2ea615ac4051d5 Mon Sep 17 00:00:00 2001 From: Karthick Duraisamy Soundararaj Date: Wed, 11 May 2016 14:28:56 -0700 Subject: [PATCH 01/12] New package for Resource Collection to address the following problems and use it to schedule and assign workers with consolidated offers per node. Adresses the following problems. Problem 1: Types of OfferResources was limited - CPU, MEM and PORTS. Each of them where either RESERVED or UNRESERVED. We treated RESERVED or UNRESERVED resources the same way. So the logic in OfferResources was simple. But we now have three different types of RESERVED, STATICALLY_RESERVED and DYNAMICALLY_RESERVED and we are starting to have special handling on each of the types of resources. One example of such code is https://github.com/mesos/storm/pull/111/files#diff-9e83ab25db3f6d262627383d8aa8f815. Problem 2: Currently, offers related logic is spread across OfferResources and RotatingMap. What we ultimately need is an interface which could be used across various parts of the framework. This commit introduces a 'storm.mesos.resources' package which is the first step in achieving the aforementioned goal. This package enables us to define an interface like 'storm.mesos.resources.Aggregator' and have various aggregators implement 'storm.mesos.resources.Aggregator' thereby abstracting the aggegation logic(including locking) from the rest of the packages. --- storm/src/main/storm/mesos/MesosNimbus.java | 763 +++++++----------- .../src/main/storm/mesos/MesosSupervisor.java | 2 +- .../DefaultReservationTypeComparator.java | 32 + .../storm/mesos/resources/OfferResources.java | 159 ++++ .../storm/mesos/resources/RangeResource.java | 214 +++++ .../mesos/resources/ReservationType.java | 24 + .../main/storm/mesos/resources/Resource.java | 45 ++ .../mesos/resources/ResourceEntries.java | 111 +++ .../storm/mesos/resources/ResourceEntry.java | 31 + .../ResourceNotAvailabeException.java | 24 + .../storm/mesos/resources/ResourceType.java | 61 ++ .../storm/mesos/resources/ScalarResource.java | 163 ++++ .../mesos/schedulers/DefaultScheduler.java | 97 ++- .../mesos/schedulers/OfferResources.java | 156 ---- .../mesos/schedulers/SchedulerUtils.java | 92 ++- .../main/storm/mesos/util/MesosCommon.java | 39 +- .../main/storm/mesos/util/PrettyProtobuf.java | 15 + .../main/storm/mesos/util/RotatingMap.java | 2 +- .../src/test/storm/mesos/MesosCommonTest.java | 43 +- .../src/test/storm/mesos/MesosNimbusTest.java | 683 ++++++++++------ .../storm/mesos/OfferRoleComparatorTest.java | 4 - storm/src/test/storm/mesos/TestUtils.java | 81 +- .../mesos/resources/OfferResourcesTest.java | 65 ++ .../schedulers/DefaultSchedulerTest.java | 170 ++-- .../mesos/schedulers/OfferResourcesTest.java | 62 -- .../mesos/schedulers/SchedulerUtilsTest.java | 1 + 26 files changed, 2007 insertions(+), 1132 deletions(-) create mode 100644 storm/src/main/storm/mesos/resources/DefaultReservationTypeComparator.java create mode 100644 storm/src/main/storm/mesos/resources/OfferResources.java create mode 100644 storm/src/main/storm/mesos/resources/RangeResource.java create mode 100644 storm/src/main/storm/mesos/resources/ReservationType.java create mode 100644 storm/src/main/storm/mesos/resources/Resource.java create mode 100644 storm/src/main/storm/mesos/resources/ResourceEntries.java create mode 100644 storm/src/main/storm/mesos/resources/ResourceEntry.java create mode 100644 storm/src/main/storm/mesos/resources/ResourceNotAvailabeException.java create mode 100644 storm/src/main/storm/mesos/resources/ResourceType.java create mode 100644 storm/src/main/storm/mesos/resources/ScalarResource.java delete mode 100644 storm/src/main/storm/mesos/schedulers/OfferResources.java create mode 100644 storm/src/test/storm/mesos/resources/OfferResourcesTest.java delete mode 100644 storm/src/test/storm/mesos/schedulers/OfferResourcesTest.java diff --git a/storm/src/main/storm/mesos/MesosNimbus.java b/storm/src/main/storm/mesos/MesosNimbus.java index 3bb244fa6..65f819b33 100644 --- a/storm/src/main/storm/mesos/MesosNimbus.java +++ b/storm/src/main/storm/mesos/MesosNimbus.java @@ -24,6 +24,7 @@ import backtype.storm.scheduler.Topologies; import backtype.storm.scheduler.TopologyDetails; import backtype.storm.scheduler.WorkerSlot; +import clojure.lang.MapEntry; import com.google.common.base.Optional; import com.google.protobuf.ByteString; import org.apache.commons.lang3.StringUtils; @@ -45,14 +46,22 @@ import org.apache.mesos.Protos.Value.Range; import org.apache.mesos.Protos.Value.Ranges; import org.apache.mesos.Protos.Value.Scalar; -import org.apache.mesos.Protos.Value.Type; import org.apache.mesos.SchedulerDriver; import org.json.simple.JSONValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.yaml.snakeyaml.Yaml; +import storm.mesos.resources.OfferResources; +import storm.mesos.resources.ReservationType; +import storm.mesos.resources.ResourceEntries; +import storm.mesos.resources.ResourceEntries.RangeResourceEntry; +import storm.mesos.resources.ResourceEntries.ScalarResourceEntry; +import storm.mesos.resources.ResourceEntry; +import storm.mesos.resources.ResourceNotAvailabeException; +import storm.mesos.resources.ResourceType; import storm.mesos.schedulers.DefaultScheduler; import storm.mesos.schedulers.IMesosStormScheduler; +import storm.mesos.schedulers.SchedulerUtils; import storm.mesos.shims.CommandLineShimFactory; import storm.mesos.shims.ICommandLineShim; import storm.mesos.shims.LocalStateShim; @@ -68,10 +77,10 @@ import java.lang.reflect.Method; import java.net.MalformedURLException; import java.net.URL; +import java.net.UnknownHostException; import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -86,7 +95,9 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import static storm.mesos.util.PrettyProtobuf.offerIDListToString; import static storm.mesos.util.PrettyProtobuf.offerMapToString; +import static storm.mesos.util.PrettyProtobuf.taskInfoListToString; public class MesosNimbus implements INimbus { public static final String CONF_EXECUTOR_URI = "mesos.executor.uri"; @@ -110,14 +121,14 @@ public class MesosNimbus implements INimbus { private static final Logger LOG = LoggerFactory.getLogger(MesosNimbus.class); private final Object _offersLock = new Object(); protected java.net.URI _configUrl; - LocalStateShim _state; - NimbusScheduler _scheduler; + private LocalStateShim _state; + private NimbusScheduler _scheduler; volatile SchedulerDriver _driver; - Timer _timer = new Timer(); - Map _conf; - Set _allowedHosts; - Set _disallowedHosts; - Optional _localFileServerPort; + private Timer _timer = new Timer(); + private Map mesosStormConf; + private Set _allowedHosts; + private Set _disallowedHosts; + private Optional _localFileServerPort; private RotatingMap _offers; private LocalFileServer _httpServer; private Map taskIDtoOfferMap; @@ -168,7 +179,8 @@ public String getHostName(Map map, String nodeId) { @Override public void prepare(Map conf, String localDir) { try { - initialize(conf, localDir); + initializeMesosStormConf(conf, localDir); + startLocalServer(); MesosSchedulerDriver driver = createMesosDriver(); @@ -187,11 +199,16 @@ public void prepare(Map conf, String localDir) { } @SuppressWarnings("unchecked") - protected void initialize(Map conf, String localDir) throws Exception { - _conf = new HashMap(); - _conf.putAll(conf); + void initializeMesosStormConf(Map conf, String localDir) { + mesosStormConf = new HashMap(); + mesosStormConf.putAll(conf); - _state = new LocalStateShim(localDir); + try { + _state = new LocalStateShim(localDir); + } catch (IOException exp) { + // TODO(ksoundararaj) : Should we exit here? + LOG.error("Encounted IOException while setting up LocalState at {} : {}", localDir, exp); + } _allowedHosts = listIntoSet((List) conf.get(CONF_MESOS_ALLOWED_HOSTS)); _disallowedHosts = listIntoSet((List) conf.get(CONF_MESOS_DISALLOWED_HOSTS)); Boolean preferReservedResources = (Boolean) conf.get(CONF_MESOS_PREFER_RESERVED_RESOURCES); @@ -209,22 +226,34 @@ protected void initialize(Map conf, String localDir) throws Exception { throw new RuntimeException("Couldn't create generated-conf dir, _generatedConfPath=" + _generatedConfPath.toString()); } - createLocalServerPort(); - setupHttpServer(); - - _conf.put(Config.NIMBUS_HOST, _configUrl.getHost()); + try { + mesosStormConf.put(Config.NIMBUS_HOST, MesosCommon.getNimbusHost(mesosStormConf)); + } catch (UnknownHostException exp) { + LOG.error("Exception while configuring nimbus host: {}", exp); + // TODO(ksoundararaj): Should we exit here? + } - File generatedConf = Paths.get(_generatedConfPath.toString(), "storm.yaml").toFile(); - Yaml yaml = new Yaml(); - FileWriter writer = new FileWriter(generatedConf); - yaml.dump(_conf, writer); + Path pathToDumpConfig = Paths.get(_generatedConfPath.toString(), "storm.yaml"); + try { + File generatedConf = pathToDumpConfig.toFile(); + Yaml yaml = new Yaml(); + FileWriter writer = new FileWriter(generatedConf); + yaml.dump(mesosStormConf, writer); + } catch (IOException exp) { + LOG.error("Could not dump generated config to {}", pathToDumpConfig); + } } + @SuppressWarnings("unchecked") + protected void startLocalServer() throws Exception { + createLocalServerPort(); + setupHttpServer(); + } public void doRegistration(final SchedulerDriver driver, Protos.FrameworkID id) { _driver = driver; _state.put(FRAMEWORK_ID, id.getValue()); - Number filterSeconds = Optional.fromNullable((Number) _conf.get(CONF_MESOS_OFFER_FILTER_SECONDS)).or(120); + Number filterSeconds = Optional.fromNullable((Number) mesosStormConf.get(CONF_MESOS_OFFER_FILTER_SECONDS)).or(120); final Protos.Filters filters = Protos.Filters.newBuilder() .setRefuseSeconds(filterSeconds.intValue()) .build(); @@ -240,7 +269,7 @@ public void expire(Protos.OfferID key, Protos.Offer val) { } ); - Number lruCacheSize = Optional.fromNullable((Number) _conf.get(CONF_MESOS_OFFER_LRU_CACHE_SIZE)).or(1000); + Number lruCacheSize = Optional.fromNullable((Number) mesosStormConf.get(CONF_MESOS_OFFER_LRU_CACHE_SIZE)).or(1000); final int intLruCacheSize = lruCacheSize.intValue(); taskIDtoOfferMap = Collections.synchronizedMap(new LinkedHashMap(intLruCacheSize + 1, .75F, true) { // This method is called just after a new entry has been added @@ -249,8 +278,8 @@ public boolean removeEldestEntry(Map.Entry eldest) { } }); - Number offerExpired = Optional.fromNullable((Number) _conf.get(Config.NIMBUS_MONITOR_FREQ_SECS)).or(10); - Number expiryMultiplier = Optional.fromNullable((Number) _conf.get(CONF_MESOS_OFFER_EXPIRY_MULTIPLIER)).or(2.5); + Number offerExpired = Optional.fromNullable((Number) mesosStormConf.get(Config.NIMBUS_MONITOR_FREQ_SECS)).or(10); + Number expiryMultiplier = Optional.fromNullable((Number) mesosStormConf.get(CONF_MESOS_OFFER_EXPIRY_MULTIPLIER)).or(2.5); _timer.scheduleAtFixedRate(new TimerTask() { @Override public void run() { @@ -306,85 +335,37 @@ public void taskLost(final TaskID taskId) { public void run() { taskIDtoOfferMap.remove(taskId); } - }, MesosCommon.getSuicideTimeout(_conf), TimeUnit.SECONDS); + }, MesosCommon.getSuicideTimeout(mesosStormConf), TimeUnit.SECONDS); } - protected void createLocalServerPort() { - Integer port = (Integer) _conf.get(CONF_MESOS_LOCAL_FILE_SERVER_PORT); + private void createLocalServerPort() { + Integer port = (Integer) mesosStormConf.get(CONF_MESOS_LOCAL_FILE_SERVER_PORT); LOG.debug("LocalFileServer configured to listen on port: {}", port); _localFileServerPort = Optional.fromNullable(port); } - protected void setupHttpServer() throws Exception { + private void setupHttpServer() throws Exception { _httpServer = new LocalFileServer(); _configUrl = _httpServer.serveDir("/generated-conf", _generatedConfPath.toString(), _localFileServerPort); LOG.info("Started HTTP server from which config for the MesosSupervisor's may be fetched. URL: {}", _configUrl); } - protected MesosSchedulerDriver createMesosDriver() throws IOException { + private MesosSchedulerDriver createMesosDriver() throws IOException { MesosSchedulerDriver driver; Credential credential; FrameworkInfo.Builder finfo = createFrameworkBuilder(); + LOG.info(String.format("Registering framework with role '%s'", finfo.getRole())); if ((credential = getCredential(finfo)) != null) { - driver = new MesosSchedulerDriver(_scheduler, finfo.build(), (String) _conf.get(CONF_MASTER_URL), credential); + driver = new MesosSchedulerDriver(_scheduler, finfo.build(), (String) mesosStormConf.get(CONF_MASTER_URL), credential); } else { - driver = new MesosSchedulerDriver(_scheduler, finfo.build(), (String) _conf.get(CONF_MASTER_URL)); + driver = new MesosSchedulerDriver(_scheduler, finfo.build(), (String) mesosStormConf.get(CONF_MASTER_URL)); } return driver; } - private void collectPorts(List offers, List portList, int maxPorts) { - for (Resource r : offers) { - if (r.getName().equals("ports")) { - for (Range range : r.getRanges().getRangeList()) { - if (portList.size() >= maxPorts) { - break; - } else { - int start = (int) range.getBegin(); - int end = (int) range.getEnd(); - for (int p = start; p <= end; p++) { - portList.add(p); - if (portList.size() >= maxPorts) { - break; - } - } - } - } - } - } - } - - /** - * Method checks if all topologies that need assignment already have supervisor running on the node where the Offer - * comes from. Required for more accurate available resource calculation where we can exclude supervisor's demand from - * the Offer. - * Unfortunately because of WorkerSlot type is not topology agnostic, we need to exclude supervisor's resources only - * in case where ALL topologies in 'allSlotsAvailableForScheduling' method satisfy condition of supervisor existence - * - * @param offer Offer - * @param existingSupervisors Supervisors which already placed on the node for the Offer - * @param topologiesMissingAssignments Topology ids required assignment - * @return Boolean value indicating supervisor existence - */ - private boolean supervisorExists( - Offer offer, Collection existingSupervisors, Set topologiesMissingAssignments) { - boolean alreadyExists = true; - for (String topologyId : topologiesMissingAssignments) { - String offerHost = offer.getHostname(); - boolean exists = false; - for (SupervisorDetails d : existingSupervisors) { - if (d.getId().equals(MesosCommon.supervisorId(offerHost, topologyId))) { - exists = true; - } - } - alreadyExists = (alreadyExists && exists); - } - return alreadyExists; - } - public boolean isHostAccepted(String hostname) { return (_allowedHosts == null && _disallowedHosts == null) || @@ -405,156 +386,8 @@ public Collection allSlotsAvailableForScheduling( } } - private OfferID findOffer(WorkerSlot worker) { - int port = worker.getPort(); - for (Offer offer : _offers.values()) { - if (offer.getHostname().equals(worker.getNodeId())) { - List r = getResourcesRange(offer.getResourcesList(), port, "ports"); - if (r != null) return offer.getId(); - } - } - // Still haven't found the slot? Maybe it's an offer we already used. - return null; - } - - protected List getResourcesScalar(final List offerResources, - final double value, - final String name) { - List resources = new ArrayList<>(); - double valueNeeded = value; - for (Resource r : offerResources) { - if (r.hasReservation()) { - // skip resources with dynamic reservations - continue; - } - if (r.getType() == Type.SCALAR && - r.getName().equals(name)) { - if (r.getScalar().getValue() > valueNeeded) { - resources.add( - r.toBuilder() - .setScalar(Scalar.newBuilder().setValue(valueNeeded)) - .build() - ); - return resources; - } else if (Math.abs(r.getScalar().getValue() - valueNeeded) < 0.0001) { // check if zero - resources.add( - r.toBuilder() - .setScalar(Scalar.newBuilder().setValue(valueNeeded)) - .build() - ); - return resources; - } else { - resources.add(r.toBuilder().build()); - valueNeeded -= r.getScalar().getValue(); - } - } - } - return resources; - } - - protected List subtractResourcesScalar(final List offerResources, - final double value, - final String name) { - List resources = new ArrayList<>(); - double valueNeeded = value; - for (Resource r : offerResources) { - if (r.hasReservation()) { - // skip reserved resources - continue; - } - if (r.getType() == Type.SCALAR && - r.getName().equals(name)) { - if (r.getScalar().getValue() > valueNeeded) { - resources.add( - r.toBuilder() - .setScalar(Scalar.newBuilder().setValue(r.getScalar().getValue() - valueNeeded)) - .build() - ); - valueNeeded = 0; - } else { - valueNeeded -= r.getScalar().getValue(); - } - } else { - resources.add(r.toBuilder().build()); - } - } - return resources; - } - - protected List getResourcesRange(final List offerResources, - final long value, - final String name) { - for (Resource r : offerResources) { - if (r.hasReservation()) { - // skip reserved resources - continue; - } - if (r.getType() == Type.RANGES && r.getName().equals(name)) { - for (Range range : r.getRanges().getRangeList()) { - if (value >= range.getBegin() && value <= range.getEnd()) { - return Arrays.asList(r.toBuilder() - .setRanges( - Ranges.newBuilder() - .addRange( - Range.newBuilder().setBegin(value).setEnd(value).build() - ).build() - ) - .build() - ); - } - } - } - } - return new ArrayList<>(); - } - - protected List subtractResourcesRange(final List offerResources, - final long value, - final String name) { - List resources = new ArrayList<>(); - for (Resource r : offerResources) { - if (r.hasReservation()) { - // skip reserved resources - continue; - } - if (r.getType() == Type.RANGES && r.getName().equals(name)) { - Resource.Builder remaining = r.toBuilder(); - Ranges.Builder ranges = Ranges.newBuilder(); - for (Range range : r.getRanges().getRangeList()) { - if (value >= range.getBegin() && value <= range.getEnd()) { - if (range.getBegin() <= value && range.getEnd() >= value) { - if (range.getBegin() == value && range.getEnd() > value) { - ranges.addRange( - Range.newBuilder().setBegin(range.getBegin() + 1).setEnd(range.getEnd()).build() - ); - } else if (range.getBegin() < value && range.getEnd() == value) { - ranges.addRange( - Range.newBuilder().setBegin(range.getBegin()).setEnd(range.getEnd() - 1).build() - ); - } else if (range.getBegin() < value && range.getEnd() > value) { - ranges.addRange( - Range.newBuilder().setBegin(range.getBegin()).setEnd(value - 1).build() - ); - ranges.addRange( - Range.newBuilder().setBegin(value + 1).setEnd(range.getEnd()).build() - ); - } else { - // skip - } - } - } else { - ranges.addRange(range.toBuilder().build()); - } - } - remaining.setRanges(ranges.build()); - if (remaining.getRanges().getRangeCount() > 0) { - resources.add(remaining.build()); - } - } else { - resources.add(r.toBuilder().build()); - } - } - return resources; + private String getLogViewerConfig() { + return String.format(" -c %s=true", MesosCommon.AUTO_START_LOGVIEWER_CONF); } /** @@ -577,288 +410,266 @@ public void assignSlots(Topologies topologies, Map offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(_offers); + Map> tasksToLaunchPerNode = getTasksToLaunch(topologies, slotsForTopologiesNeedingAssignments, offerResourcesPerNode); - /** - * @param topologies - Information about all submitted topologies - * @param slotsForTopologiesNeedingAssignments - A map of topology name and collection of worker slots that are assigned to the topology - * that need assignments - */ - protected void computeLaunchList(Topologies topologies, Map> slotsForTopologiesNeedingAssignments) { - Map> toLaunch = new HashMap<>(); - // For every topology that needs assignment - for (String topologyId : slotsForTopologiesNeedingAssignments.keySet()) { - Map> offerIDtoWorkerSlotMap = new HashMap<>(); - // Find a corresponding offer for every slot that needs to be launched - for (WorkerSlot slot : slotsForTopologiesNeedingAssignments.get(topologyId)) { - OfferID id = findOffer(slot); - if (!offerIDtoWorkerSlotMap.containsKey(id)) { - offerIDtoWorkerSlotMap.put(id, new ArrayList()); + for (String node : tasksToLaunchPerNode.keySet()) { + List offerIDList = offerResourcesPerNode.get(node).getOfferIDList(); + List taskInfoList = tasksToLaunchPerNode.get(node); + + LOG.info("Using offerIDs: " + offerIDListToString(offerIDList) + " on host: " + node + " to launch tasks: " + taskInfoListToString(taskInfoList)); + + _driver.launchTasks(offerIDList, taskInfoList); + for (OfferID offerID: offerIDList) { + _offers.remove(offerID); } - /* - * Find offer could return "null" or an OfferID - * If a port that is specified in the workerSlot is found, findOffer returns the OfferID associated with the port - * If a port is not found, findOffer returns null - */ - offerIDtoWorkerSlotMap.get(id).add(slot); } + } + } + + private Resource createMesosScalarResource(ResourceType resourceType, ScalarResourceEntry scalarResourceEntry) { + return Resource.newBuilder() + .setName(resourceType.toString()) + .setType(Protos.Value.Type.SCALAR) + .setScalar(Scalar.newBuilder().setValue(scalarResourceEntry.getValue())) + .build(); + + } - // At this point, we have a map of OfferID and WorkerSlot in the form of offerIDtoWorkerSlotMap - // Note that at this point, we only know that the ports are available on the node. We still have to - // find cpu and memory in order to launch the workers - for (OfferID id : offerIDtoWorkerSlotMap.keySet()) { - computeResourcesForSlot(_offers, topologies, toLaunch, topologyId, offerIDtoWorkerSlotMap, id); + private List createMesosScalarResource(ResourceType resourceType, List scalarResourceEntryList) { + List retVal = new ArrayList<>(); + ScalarResourceEntry scalarResourceEntry = null; + + for (ResourceEntry resourceEntry : scalarResourceEntryList) { + scalarResourceEntry = (ScalarResourceEntry) resourceEntry; + Resource.Builder resourceBuilder = Resource.newBuilder() + .setName(resourceType.toString()) + .setType(Protos.Value.Type.SCALAR) + .setScalar(Scalar.newBuilder().setValue(scalarResourceEntry.getValue())); + if (resourceEntry.getReservationType() != null && resourceEntry.getReservationType().equals(ReservationType.STATICALLY_RESERVED)) { + resourceBuilder.setRole(MesosCommon.getRole(mesosStormConf)); } + retVal.add(resourceBuilder.build()); } + return retVal; + } - for (OfferID id : toLaunch.keySet()) { - List tasks = toLaunch.get(id); - List launchList = new ArrayList<>(); + private Resource createMesosRangeResource(ResourceType resourceType, RangeResourceEntry rangeResourceEntry) { + Ranges.Builder rangesBuilder = Ranges.newBuilder(); + Range rangeBuilder = Range.newBuilder() + .setBegin(rangeResourceEntry.getBegin()) + .setEnd(rangeResourceEntry.getEnd()).build(); + rangesBuilder.addRange(rangeBuilder); + + Resource.Builder resourceBuilder = Resource.newBuilder() + .setName(resourceType.toString()) + .setType(Protos.Value.Type.RANGES) + .setRanges(rangesBuilder.build()); + if (rangeResourceEntry.getReservationType() != null && rangeResourceEntry.getReservationType().equals(ReservationType.STATICALLY_RESERVED)) { + resourceBuilder.setRole(MesosCommon.getRole(mesosStormConf)); + } + return resourceBuilder.build(); + } - LOG.info("Launching tasks for offerId: {} : {}", id.getValue(), launchTaskListToString(tasks)); - for (LaunchTask t : tasks) { - launchList.add(t.getTask()); - taskIDtoOfferMap.put(t.getTask().getTaskId(), t.getOffer()); - } + String getFullConfigUri() { + try { + return new URL(_configUrl.toURL(), + _configUrl.getPath() + "/storm.yaml").toString(); + } catch (MalformedURLException e) { + throw new RuntimeException(e); + } + } - List launchOffer = new ArrayList<>(); - launchOffer.add(id); - _driver.launchTasks(launchOffer, launchList); - _offers.remove(id); + private ExecutorInfo.Builder getExecutorInfoBuilder(TopologyDetails details, String executorDataStr, + String executorName, + List executorCpuResources, + List executorMemResources, + List executorPortsResources, + String extraConfig) { + String configUri; + + configUri = getFullConfigUri(); + + ExecutorInfo.Builder executorInfoBuilder = ExecutorInfo.newBuilder(); + + executorInfoBuilder + .setName(executorName) + .setExecutorId(ExecutorID.newBuilder().setValue(details.getId())) + .setData(ByteString.copyFromUtf8(executorDataStr)) + .addAllResources(executorCpuResources) + .addAllResources(executorMemResources) + .addAllResources(executorPortsResources); + + ICommandLineShim commandLineShim = CommandLineShimFactory.makeCommandLineShim(_container.isPresent(), extraConfig); + if (_container.isPresent()) { + executorInfoBuilder.setCommand(CommandInfo.newBuilder() + .addUris(URI.newBuilder().setValue(configUri)) + .setValue(commandLineShim.getCommandLine(details.getId()))) + .setContainer(ContainerInfo.newBuilder() + .setType(ContainerInfo.Type.DOCKER) + .setDocker(ContainerInfo.DockerInfo.newBuilder() + .setImage(_container.get()) + .setNetwork(ContainerInfo.DockerInfo.Network.HOST) + .setForcePullImage(true) + .build() + ).build()); + } else { + executorInfoBuilder.setCommand(CommandInfo.newBuilder() + .addUris(URI.newBuilder().setValue((String) mesosStormConf.get(CONF_EXECUTOR_URI))) + .addUris(URI.newBuilder().setValue(configUri)) + .setValue(commandLineShim.getCommandLine(details.getId()))); } + + return executorInfoBuilder; } - /** - * Considering the way this method is invoked - computeResourcesForSlot(_offers, topologies, toLaunch, topologyId, offerIDtoWorkerSlotMap, id) - - * the following params can be removed/refactored. - * 1) offers - Method could just use the _offers private variable directly. - * 2) toLaunch - This should be a return value - * 3) offerId/offerIDtoWorkerSlotMap - Passing both of these params is redundant - * @param offers - All available offers. - * @param topologies - Information about all submitted topologies - * @param toLaunch - A map of offerID and list of tasks that needs to be launched using the OfferID - * @param topologyId - TopologyId for which we are computing resources - * @param offerIDtoWorkerSlotMap - Map of OfferID that contains ports for the list of workerSlots - * @param offerId - One of the keys in offerIDtoWorkerSlotMap - */ - protected void computeResourcesForSlot(final RotatingMap offers, - Topologies topologies, - Map> toLaunch, - String topologyId, - Map> offerIDtoWorkerSlotMap, - OfferID offerId) { - boolean usingExistingOffer = false; - boolean subtractedExecutorResources = false; - Offer offer = offers.get(offerId); - List workerSlots = offerIDtoWorkerSlotMap.get(offerId); - - // Note: As of writing this comment, the workerSlots belong to same topology - for (WorkerSlot slot : workerSlots) { - TopologyDetails details = topologies.getById(topologyId); - String workerPrefix = ""; - if (_conf.containsKey(MesosCommon.WORKER_NAME_PREFIX)) { - workerPrefix = MesosCommon.getWorkerPrefix(_conf, details); - } - TaskID taskId = TaskID.newBuilder() - .setValue(MesosCommon.taskId(workerPrefix + slot.getNodeId(), slot.getPort())) - .build(); - - // taskIDtoOfferMap is unnecessary - // 1. OfferID is usable only once - That is if we ask mesos to launch task(s) on an offerID, - // mesos returns remnants as a different offer with a completely different offerID - // 2. If MesosNimbus is restarted, all this information is lost. - // 3. By not clearing taskIDtoOfferMap at the end of this method, we are only wasting memory - // The following if condition to check taskIDtoOfferMap is unnecessary - // 1. This function is invoked only once per slot. - // 2. If the function is invoked a second time for the same slot, then it's either because the task - // was finished or killed. Either way, the old offerId we used to launch the worker is invalid - if ((offerId == null || offer == null) && taskIDtoOfferMap.containsKey(taskId)) { - offer = taskIDtoOfferMap.get(taskId); - if (offer != null) { - offerId = offer.getId(); - usingExistingOffer = true; + public Map> getTasksToLaunch(Topologies topologies, + Map> slots, + Map offerResourcesPerNode) { + Map> tasksToLaunchPerNode = new HashMap<>(); + + for (String topologyId : slots.keySet()) { + Collection slotList = slots.get(topologyId); + TopologyDetails topologyDetails = topologies.getById(topologyId); + Set hostsUsedSoFar = new HashSet<>(); + + double workerCpu = MesosCommon.topologyWorkerCpu(mesosStormConf, topologyDetails); + double workerMem = MesosCommon.topologyWorkerMem(mesosStormConf, topologyDetails); + double executorCpu = MesosCommon.executorCpu(mesosStormConf); + double executorMem = MesosCommon.executorMem(mesosStormConf); + double requiredCpu = workerCpu; + double requiredMem = workerMem; + + for (WorkerSlot slot : slotList) { + String workerHost = slot.getNodeId(); + Long workerPort = Long.valueOf(slot.getPort()); + + Boolean subtractExecutorResources = !hostsUsedSoFar.contains(workerHost); + hostsUsedSoFar.add(workerHost); + + OfferResources offerResources = offerResourcesPerNode.get(slot.getNodeId()); + String workerPrefix = ""; + if (mesosStormConf.containsKey(MesosCommon.WORKER_NAME_PREFIX)) { + workerPrefix = MesosCommon.getWorkerPrefix(mesosStormConf, topologyDetails); } - } - // The following way of finding resources for the slot is bad! - // Suppose we have the following offers: - // o1 - { host: h1 ports : 31000-32000 } - // o2 - { host: h1 mem : 30000 cpu : 24 } - // because the offers are fragmented, they are useless - // 1. At this point a worker slot "h1-3000" associated with o1 for instance wouldn't - // be launched because it doesn't have mem and cpu - // 2. o2 is useless because it doesn't have any ports and therefore won't be used at all - if (offerId != null && offer != null) { - // The fact that we are here means that this offer has a port. We need to find if it also has - // enough memory and cpu to launch the worker. - if (!toLaunch.containsKey(offerId)) { - toLaunch.put(offerId, new ArrayList()); + if (hostsUsedSoFar.contains(workerHost)) { + requiredCpu += executorCpu; + requiredMem += executorMem; } - double workerCpu = MesosCommon.topologyWorkerCpu(_conf, details); - double workerMem = MesosCommon.topologyWorkerMem(_conf, details); - double executorCpu = MesosCommon.executorCpu(_conf); - double executorMem = MesosCommon.executorMem(_conf); Map executorData = new HashMap(); - executorData.put(MesosCommon.SUPERVISOR_ID, MesosCommon.supervisorId(slot.getNodeId(), details.getId())); + executorData.put(MesosCommon.SUPERVISOR_ID, MesosCommon.supervisorId(slot.getNodeId(), topologyDetails.getId())); executorData.put(MesosCommon.ASSIGNMENT_ID, workerPrefix + slot.getNodeId()); - Offer.Builder newBuilder = Offer.newBuilder(); - newBuilder.mergeFrom(offer); - newBuilder.clearResources(); + String topologyAndNodeId = topologyDetails.getId() + " | " + slot.getNodeId(); + String executorName = "storm-supervisor | " + topologyAndNodeId; + String taskName = "storm-worker | " + topologyAndNodeId + ":" + slot.getPort(); + String executorDataStr = JSONValue.toJSONString(executorData); - Offer.Builder existingBuilder = Offer.newBuilder(); - existingBuilder.mergeFrom(offer); - existingBuilder.clearResources(); - String extraConfig = ""; + boolean autostartLogviewer = !subtractExecutorResources && MesosCommon.autoStartLogViewer(mesosStormConf); - List offerResources = new ArrayList<>(); - offerResources.addAll(offer.getResourcesList()); - // Prefer reserved resources? - if (_preferReservedResources) { - Collections.sort(offerResources, new ResourceRoleComparator()); + if (!SchedulerUtils.isFit(mesosStormConf, offerResources, topologyDetails, workerPort, !subtractExecutorResources, autostartLogviewer)) { + LOG.error(String.format("Unable to launch worker %s. Required cpu: %f, Required mem: %f. Available OfferResources : %s", + workerHost, requiredCpu, requiredMem, offerResources)); + continue; } - List executorCpuResources = getResourcesScalar(offerResources, executorCpu, "cpus"); - List executorMemResources = getResourcesScalar(offerResources, executorMem, "mem"); - List executorPortsResources = null; - - // Question(ksoundararaj): - // Shouldn't we be validating executorCpuResources and executorCpuResources to ensure they aren't - // empty or less than what was requested? - if (!subtractedExecutorResources) { - offerResources = subtractResourcesScalar(offerResources, executorCpu, "cpus"); - offerResources = subtractResourcesScalar(offerResources, executorMem, "mem"); - subtractedExecutorResources = true; - } - List workerCpuResources = getResourcesScalar(offerResources, workerCpu, "cpus"); - offerResources = subtractResourcesScalar(offerResources, workerCpu, "cpus"); - List workerMemResources = getResourcesScalar(offerResources, workerMem, "mem"); - offerResources = subtractResourcesScalar(offerResources, workerMem, "mem"); - List workerPortsResources = getResourcesRange(offerResources, slot.getPort(), "ports"); - offerResources = subtractResourcesRange(offerResources, slot.getPort(), "ports"); - - // Find port for the logviewer - if (!subtractedExecutorResources && MesosCommon.startLogViewer(_conf)) { - List portList = new ArrayList<>(); - collectPorts(offerResources, portList, 1); - int port = Optional.fromNullable((Number) _conf.get(Config.LOGVIEWER_PORT)).or(8000).intValue(); - executorPortsResources = getResourcesRange(offerResources, port, "ports"); - if (!executorPortsResources.isEmpty()) { - // Was the port available? - extraConfig += String.format(" -c %s=true", MesosCommon.AUTO_START_LOGVIEWER_CONF); - offerResources = subtractResourcesRange(offerResources, port, "ports"); - } - } + List executorCpuResources = new ArrayList<>(); + List executorMemResources = new ArrayList<>(); + List executorPortResources = new ArrayList<>(); + List workerCpuResources = new ArrayList<>(); + List workerMemResources = new ArrayList<>(); + List workerPortResources = new ArrayList<>(); + - // Set the MesosSupervisor's storm.local.dir value. By default it is set to "storm-local", - // which is the same as the storm-core default, which is interpreted relative to the pwd of - // the storm daemon (the mesos executor sandbox in the case of the MesosSupervisor). - // - // If this isn't done, then MesosSupervisor inherits MesosNimbus's storm.local.dir config. - // That is fine if the MesosNimbus storm.local.dir config is just the default too ("storm-local"), - // since the mesos executor sandbox is isolated per-topology. - // However, if the MesosNimbus storm.local.dir is specialized (e.g., /var/run/storm-local), then - // this will cause problems since multiple topologies on the same host would use the same - // storm.local.dir and thus interfere with each other's state. And *that* leads to one topology's - // supervisor killing the workers of every other topology on the same host (due to interference - // in the stored worker assignments). - // - // Note that you *can* force the MesosSupervisor to use a specific directory by setting - // the "mesos.supervisor.storm.local.dir" variable in the MesosNimbus's storm.yaml. - extraConfig += String.format(" -c storm.local.dir=%s", MesosCommon.getSupervisorStormLocalDir(_conf)); - - Offer remainingOffer = existingBuilder.addAllResources(offerResources).build(); - - // Update the remaining offer list - offers.put(offerId, remainingOffer); - - // At this point, hopefully, we have all the resources we need. - String configUri; try { - configUri = new URL(_configUrl.toURL(), _configUrl.getPath() + "/storm.yaml").toString(); - } catch (MalformedURLException e) { - throw new RuntimeException(e); - } + List scalarResourceEntryList = null; + List rangeResourceEntryList = null; + + if (subtractExecutorResources) { + scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.CPU, new ScalarResourceEntry(executorCpu)); + executorCpuResources.addAll(createMesosScalarResource(ResourceType.CPU, scalarResourceEntryList)); + scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.MEM, new ScalarResourceEntry(executorMem)); + executorMemResources.addAll(createMesosScalarResource(ResourceType.MEM, scalarResourceEntryList)); + } else { + // TODO(ksoundararaj): Do we need to ensure consistent roles for all executors belonging to same topology? + // If so, how do we do that? + executorCpuResources.add(createMesosScalarResource(ResourceType.CPU, new ScalarResourceEntry(executorCpu))); + executorMemResources.add(createMesosScalarResource(ResourceType.MEM, new ScalarResourceEntry(executorMem))); + } - String delimiter = MesosCommon.getMesosComponentNameDelimiter(_conf, details); - String topologyAndNodeId = String.format("%s%s%s", details.getId(), delimiter, slot.getNodeId()); - String executorName = String.format("storm-supervisor%s%s", delimiter, topologyAndNodeId); - String taskName = String.format("storm-worker%s%s:%d", delimiter, topologyAndNodeId, slot.getPort()); - String executorDataStr = JSONValue.toJSONString(executorData); - ExecutorInfo.Builder executorInfoBuilder = ExecutorInfo.newBuilder(); - executorInfoBuilder - .setName(executorName) - .setExecutorId(ExecutorID.newBuilder().setValue(details.getId())) - .setData(ByteString.copyFromUtf8(executorDataStr)) - .addAllResources(executorCpuResources) - .addAllResources(executorMemResources); - if (executorPortsResources != null) { - executorInfoBuilder.addAllResources(executorPortsResources); - } - ICommandLineShim commandLineShim = CommandLineShimFactory.makeCommandLineShim(_container.isPresent(), extraConfig); - if (_container.isPresent()) { - executorInfoBuilder - .setCommand(CommandInfo.newBuilder() - .addUris(URI.newBuilder().setValue(configUri)) - .setValue(commandLineShim.getCommandLine(details.getId()))) - .setContainer( - ContainerInfo.newBuilder() - .setType(ContainerInfo.Type.DOCKER) - .setDocker( - ContainerInfo.DockerInfo.newBuilder() - .setImage(_container.get()) - .setNetwork(ContainerInfo.DockerInfo.Network.HOST) - .setForcePullImage(true) - .build() - ) - .build() - ); - } else { - executorInfoBuilder - .setCommand(CommandInfo.newBuilder() - .addUris(URI.newBuilder().setValue((String) _conf.get(CONF_EXECUTOR_URI))) - .addUris(URI.newBuilder().setValue(configUri)) - .setValue(commandLineShim.getCommandLine(details.getId()))); + /** + * Autostart logviewer cannot be supported. If we try to fetch a random port P + * from the list of available ports and assign it to the logviewer, its possible that + * one of the workers is already assigned to that port. So I am not supporting auto starting + * logviewer at this point + * + if (MesosCommon.autoStartLogViewer(mesosStormConf)) { + Long logViewerPort = offerResources.reserveRangeResource(ResourceType.PORTS, 1).get(0); + List logViewerPortList = new ArrayList<>(); + logViewerPortList.add(new RangeResourceEntry(logViewerPort, logViewerPort)); + executorPortResources.add(createMesosRangeResource(ResourceType.PORTS, logViewerPortList)); + } */ + + scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.CPU, new ScalarResourceEntry(workerCpu)); + workerCpuResources.addAll(createMesosScalarResource(ResourceType.CPU, scalarResourceEntryList)); + scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.MEM, new ScalarResourceEntry(workerMem)); + workerCpuResources.addAll(createMesosScalarResource(ResourceType.MEM, scalarResourceEntryList)); + rangeResourceEntryList = offerResources.reserveAndGet(ResourceType.PORTS, new RangeResourceEntry(workerPort, workerPort)); + for (ResourceEntry resourceEntry : rangeResourceEntryList) { + workerCpuResources.add(createMesosRangeResource(ResourceType.PORTS, (RangeResourceEntry) resourceEntry)); + } + } catch (ResourceNotAvailabeException rexp) { + LOG.warn("Unable to launch worker %s. Required cpu: %f, Required mem: %f. Available OfferResources : %s", + workerHost, requiredCpu, requiredMem, offerResources); + continue; } - LOG.info("Launching task with Mesos Executor data: < {} >", executorDataStr); - TaskInfo task = TaskInfo.newBuilder() - .setName(taskName) - .setTaskId(taskId) - .setSlaveId(offer.getSlaveId()) - .setExecutor(executorInfoBuilder.build()) - .addAllResources(workerCpuResources) - .addAllResources(workerMemResources) - .addAllResources(workerPortsResources) - .build(); - - Offer newOffer = offer.toBuilder() - .addAllResources(task.getResourcesList()).build(); - LOG.debug("Launching task: {}", task.toString()); + String extraConfig = ""; + if (autostartLogviewer) { + extraConfig = getLogViewerConfig(); + } - toLaunch.get(offerId).add(new LaunchTask(task, newOffer)); - } + ExecutorInfo.Builder executorInfoBuilder = getExecutorInfoBuilder(topologyDetails, executorDataStr, executorName, executorCpuResources, + executorMemResources, executorPortResources, extraConfig); + TaskID taskId = TaskID.newBuilder() + .setValue(MesosCommon.taskId(slot.getNodeId(), slot.getPort())) + .build(); - if (usingExistingOffer) { - _driver.killTask(taskId); + TaskInfo task = TaskInfo.newBuilder() + .setTaskId(taskId) + .setName(taskName) + .setSlaveId(offerResources.getSlaveID()) + .setExecutor(executorInfoBuilder.build()) + .addAllResources(workerCpuResources) + .addAllResources(workerMemResources) + .addAllResources(workerPortResources) + .build(); + + List taskInfoList = tasksToLaunchPerNode.get(slot.getNodeId()); + if (taskInfoList == null) { + taskInfoList = new ArrayList<>(); + tasksToLaunchPerNode.put(slot.getNodeId(), taskInfoList); + } + taskInfoList.add(task); } } + + return tasksToLaunchPerNode; } private FrameworkInfo.Builder createFrameworkBuilder() throws IOException { - Number failoverTimeout = Optional.fromNullable((Number) _conf.get(CONF_MASTER_FAILOVER_TIMEOUT_SECS)).or(24 * 7 * 3600); - String role = Optional.fromNullable((String) _conf.get(CONF_MESOS_ROLE)).or("*"); - Boolean checkpoint = Optional.fromNullable((Boolean) _conf.get(CONF_MESOS_CHECKPOINT)).or(false); - String frameworkName = Optional.fromNullable((String) _conf.get(CONF_MESOS_FRAMEWORK_NAME)).or("Storm!!!"); + Number failoverTimeout = Optional.fromNullable((Number) mesosStormConf.get(CONF_MASTER_FAILOVER_TIMEOUT_SECS)).or(24 * 7 * 3600); + String role = Optional.fromNullable((String) mesosStormConf.get(CONF_MESOS_ROLE)).or("*"); + Boolean checkpoint = Optional.fromNullable((Boolean) mesosStormConf.get(CONF_MESOS_CHECKPOINT)).or(false); + String frameworkName = Optional.fromNullable((String) mesosStormConf.get(CONF_MESOS_FRAMEWORK_NAME)).or("Storm!!!"); FrameworkInfo.Builder finfo = FrameworkInfo.newBuilder() .setName(frameworkName) @@ -895,8 +706,8 @@ private Credential getCredential(FrameworkInfo.Builder finfo) { Credential credential = null; - String principal = Optional.fromNullable((String) _conf.get(CONF_MESOS_PRINCIPAL)).orNull(); - String secretFilename = Optional.fromNullable((String) _conf.get(CONF_MESOS_SECRET_FILE)).orNull(); + String principal = Optional.fromNullable((String) mesosStormConf.get(CONF_MESOS_PRINCIPAL)).orNull(); + String secretFilename = Optional.fromNullable((String) mesosStormConf.get(CONF_MESOS_SECRET_FILE)).orNull(); if (principal != null) { finfo.setPrincipal(principal); diff --git a/storm/src/main/storm/mesos/MesosSupervisor.java b/storm/src/main/storm/mesos/MesosSupervisor.java index 6459873d6..ab287b17b 100644 --- a/storm/src/main/storm/mesos/MesosSupervisor.java +++ b/storm/src/main/storm/mesos/MesosSupervisor.java @@ -158,7 +158,7 @@ public void killedWorker(int port) { } protected boolean startLogViewer(Map conf) { - return MesosCommon.startLogViewer(conf); + return MesosCommon.autoStartLogViewer(conf); } class StormExecutor implements Executor { diff --git a/storm/src/main/storm/mesos/resources/DefaultReservationTypeComparator.java b/storm/src/main/storm/mesos/resources/DefaultReservationTypeComparator.java new file mode 100644 index 000000000..95393b35d --- /dev/null +++ b/storm/src/main/storm/mesos/resources/DefaultReservationTypeComparator.java @@ -0,0 +1,32 @@ +/** + * 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 storm.mesos.resources; + +import java.util.Comparator; + +public class DefaultReservationTypeComparator implements Comparator { + + public int compare(ReservationType left, ReservationType right) { + if (left.equals(right)) { + return 0; + } else if (left.equals(ReservationType.UNRESERVED)) { + return 1; + } + return -1; + } +} diff --git a/storm/src/main/storm/mesos/resources/OfferResources.java b/storm/src/main/storm/mesos/resources/OfferResources.java new file mode 100644 index 000000000..ceed063fc --- /dev/null +++ b/storm/src/main/storm/mesos/resources/OfferResources.java @@ -0,0 +1,159 @@ +/** + * 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 storm.mesos.resources; + +import org.apache.mesos.Protos; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import storm.mesos.util.PrettyProtobuf; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +// TODO(ksoundararaj): a better name? - OfferAggregator? +public class OfferResources { + + private final Logger log = LoggerFactory.getLogger(OfferResources.class); + private Map availableResources; + + private List offerList = new ArrayList(); + + private final String hostName; + + private Protos.SlaveID slaveID; + + private void initializeAvailableResources() { + availableResources = new HashMap<>(); + availableResources.put(ResourceType.CPU, new ScalarResource(ResourceType.CPU)); + availableResources.put(ResourceType.MEM, new ScalarResource(ResourceType.MEM)); + availableResources.put(ResourceType.PORTS, new RangeResource(ResourceType.PORTS)); + } + + public OfferResources(String hostName) { + this.hostName = hostName; + initializeAvailableResources(); + } + + public OfferResources(Protos.Offer offer) { + initializeAvailableResources(); + if (slaveID == null) { + this.slaveID = offer.getSlaveId(); + } + this.hostName = offer.getHostname(); + add(offer); + } + + public String getHostName() { + return hostName; + } + + public void add(Protos.Offer offer) { + offerList.add(offer); + + for (Protos.Resource r : offer.getResourcesList()) { + if (r.hasReservation()) { + // skip resources with dynamic reservations + continue; + } + + ResourceType resourceType = ResourceType.of(r.getName()); + log.info("Processing Resource {}", r.getName()); + + ReservationType reservationType = (r.getRole().equals("*")) ? ReservationType.UNRESERVED : ReservationType.STATICALLY_RESERVED; + + switch (resourceType) { + case CPU: + case MEM: + ResourceEntries.ScalarResourceEntry scalarResourceEntry = new ResourceEntries.ScalarResourceEntry(reservationType, r.getScalar().getValue()); + availableResources.get(resourceType).add(scalarResourceEntry, reservationType); + break; + case PORTS: + for (Protos.Value.Range range : r.getRanges().getRangeList()) { + ResourceEntries.RangeResourceEntry rangeResourceEntry = new ResourceEntries.RangeResourceEntry(reservationType, range.getBegin(), range.getEnd()); + availableResources.get(resourceType).add(rangeResourceEntry, reservationType); + } + break; + case DISK: + break; + default: + log.warn(String.format("Found unsupported resourceType '%s' while adding offer %s", resourceType, PrettyProtobuf.offerToString(offer))); + } + } + } + + public boolean isAvaliable(ResourceType resourceType, ResourceEntry resource) { + return availableResources.get(resourceType).isAvailable(resource); + } + + public boolean isAvaliable(ResourceType resourceType, ReservationType reservationType, ResourceEntry resource) { + return availableResources.get(resourceType).isAvailable(resource, reservationType); + } + + public List getAllAvailableResources(ResourceType resourceType) { + return availableResources.get(resourceType).getAllAvailableResources(); + } + + public List getAllAvailableResources(ResourceType resourceType, ReservationType reservationType) { + return availableResources.get(resourceType).getAllAvailableResources(reservationType); + } + + public void reserve(ResourceType resourceType, ResourceEntry resource) throws ResourceNotAvailabeException { + if (availableResources.get(resourceType).isAvailable(resource)) { + availableResources.get(resourceType).removeAndGet(resource); + } + } + + public List reserveAndGet(ResourceType resourceType, ResourceEntry resource) throws ResourceNotAvailabeException { + if (availableResources.get(resourceType).isAvailable(resource)) { + return availableResources.get(resourceType).removeAndGet(resource); + } + return new ArrayList<>(); + } + + public List reserveAndGet(ResourceType resourceType, ReservationType reservationType, ResourceEntry resource) throws ResourceNotAvailabeException { + if (availableResources.get(resourceType).isAvailable(resource, reservationType)) { + availableResources.get(resourceType).removeAndGet(resource, reservationType); + } + return new ArrayList<>(); + } + + public List getOfferList() { + return offerList; + } + + public List getOfferIDList() { + List offerIDList = new ArrayList<>(); + for (Protos.Offer offer: offerList) { + offerIDList.add(offer.getId()); + } + return offerIDList; + } + + public Protos.SlaveID getSlaveID() { + return slaveID; + } + + @Override + public String toString() { + return String.format("cpu : %s, memory: %s, ports : %s", availableResources.get(ResourceType.CPU), + availableResources.get(ResourceType.MEM), availableResources.get(ResourceType.PORTS)); + } +} + diff --git a/storm/src/main/storm/mesos/resources/RangeResource.java b/storm/src/main/storm/mesos/resources/RangeResource.java new file mode 100644 index 000000000..577d6112d --- /dev/null +++ b/storm/src/main/storm/mesos/resources/RangeResource.java @@ -0,0 +1,214 @@ +/** + * 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 storm.mesos.resources; + +import org.apache.commons.lang3.StringUtils; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import static storm.mesos.resources.ResourceEntries.RangeResourceEntry; + +public final class RangeResource implements Resource { + + public final ResourceType resourceType; + private final Map> availableResourcesByReservationType; + + public RangeResource(ResourceType resourceType) { + this.resourceType = resourceType; + availableResourcesByReservationType = new TreeMap<>(new DefaultReservationTypeComparator()); + for (ReservationType reservationType : ReservationType.values()) { + availableResourcesByReservationType.put(reservationType, new ArrayList()); + } + } + + @Override + public boolean isAvailable(RangeResourceEntry rangeResourceEntry) { + Long beginValue = rangeResourceEntry.getBegin(); + Long endValue = rangeResourceEntry.getEnd(); + + for (List rangeList : availableResourcesByReservationType.values()) { + for (RangeResourceEntry r : rangeList) { + if (beginValue >= r.getBegin() && endValue <= r.getEnd()) + return true; + } + } + return false; + } + + @Override + public boolean isAvailable(RangeResourceEntry rangeResourceEntry, ReservationType reservationType) { + Long beginValue = rangeResourceEntry.getBegin(); + Long endValue = rangeResourceEntry.getEnd(); + + for (RangeResourceEntry r : availableResourcesByReservationType.get(reservationType)) { + if (beginValue >= r.getBegin() && endValue <= r.getEnd()) + return true; + } + return false; + } + + @Override + public List getAllAvailableResources() { + List retVal = new ArrayList<>(); + for (ReservationType reservationType : ReservationType.values()) { + retVal.addAll(getAllAvailableResources(reservationType)); + } + return retVal; + } + + @Override + public List getAllAvailableResources(final ReservationType reservationType) { + return availableResourcesByReservationType.get(reservationType); + } + + @Override + public void add(RangeResourceEntry rangeResourceEntry, ReservationType reservationType) { + availableResourcesByReservationType.get(reservationType).add(rangeResourceEntry); + } + + /** + * Remove/Reserve range from available ranges. When resources of several reservation types ({@link storm.mesos.resources.ReservationType}) + * are available, the priority of reservation type removed is governed by {@link storm.mesos.resources.DefaultReservationTypeComparator} + * {@param rangeResourceEntry} range resource to removeAndGet + */ + @Override + public List removeAndGet(RangeResourceEntry rangeResourceEntry) throws ResourceNotAvailabeException { + if (isAvailable(rangeResourceEntry)) { + return removeAndGet(availableResourcesByReservationType.keySet(), rangeResourceEntry); + } + + String message = String.format("ResourceType '%s' is not available. Requested value: %s Available: %s", + resourceType, rangeResourceEntry, toString()); + throw new ResourceNotAvailabeException(message); + } + + + /** + * Remove/Reserve range from available ranges. + * {@param rangeResourceEntry} range resource to removeAndGet + * {@parm reservationType} reservation type of resource that needs to be removed. If the resource represented by rangeResourceEntry + * of the reservation type specied by this parameter is not available, then {@link storm.mesos.resources.ResourceNotAvailabeException} + * is thrown + */ + @Override + public List removeAndGet(RangeResourceEntry rangeResourceEntry, ReservationType reservationType) throws ResourceNotAvailabeException { + + if (isAvailable(rangeResourceEntry, reservationType)) { + List reservationTypeList = new ArrayList<>(); + reservationTypeList.add(reservationType); + return removeAndGet(reservationTypeList, rangeResourceEntry); + } + + String message = String.format("ResourceType '%s' of reservationType '%s' is not available. Requested value: %s Available: %s", + resourceType, reservationType.toString(), rangeResourceEntry.toString(), toString(availableResourcesByReservationType.get(reservationType))); + throw new ResourceNotAvailabeException(message); + } + + /** + * Remove/Reserve range from available ranges + * {@param rangeResourceEntry} range resource to removeAndGet + * {@param reservationTypeComparator} comparator like {@link storm.mesos.resources.DefaultReservationTypeComparator} + * to determine the priority of reservation types. When resources of several reservation types ({@link storm.mesos.resources.ReservationType}) + * are available, the priority of reservation type removed is governed by {@link storm.mesos.resources.DefaultReservationTypeComparator} + */ + @Override + public List removeAndGet(RangeResourceEntry rangeResourceEntry, Comparator reservationTypeCompartor) throws ResourceNotAvailabeException { + if (isAvailable(rangeResourceEntry)) { + List reservationTypeList = Arrays.asList(ReservationType.values()); + Collections.sort(reservationTypeList, reservationTypeCompartor); + return removeAndGet(reservationTypeList, rangeResourceEntry); + } + + String message = String.format("ResourceType '%s' is not available. Requested value: %s Available: %s", + resourceType, rangeResourceEntry, toString()); + throw new ResourceNotAvailabeException(message); + } + + public String toString(List ranges) { + List rangeStrings = new ArrayList<>(); + for (RangeResourceEntry range : ranges) { + String beginStr = String.valueOf(range.getBegin()); + String endStr = String.valueOf(range.getEnd()); + /* + * A Range representing a single number still has both Range.begin + * and Range.end populated, but they are set to the same value. + * In that case we just return "N" instead of "N-N". + */ + if (range.getBegin() == range.getEnd()) { + rangeStrings.add(beginStr); + } else { + rangeStrings.add(String.format("%s-%s", beginStr, endStr)); + } + } + return String.format("[%s]", StringUtils.join(rangeStrings, ",")); + } + + public String toString(ReservationType reservationType) { + return String.format("%s : %s", reservationType, toString(availableResourcesByReservationType.get(reservationType))); + } + + public String toString() { + List resourceRanges = new ArrayList<>(); + for (Map.Entry> entry : availableResourcesByReservationType.entrySet()) { + resourceRanges.addAll(entry.getValue()); + } + return toString(resourceRanges); + } + + private List removeAndGet(Collection reservationTypes, RangeResourceEntry rangeResourceEntry) { + List removedResources = new ArrayList<>(); + Long beginValue = rangeResourceEntry.getBegin(); + Long endValue = rangeResourceEntry.getEnd(); + + for (ReservationType reservationType : reservationTypes) { + List rangeResourceEntryList = availableResourcesByReservationType.get(reservationType); + for (int i = 0; i < rangeResourceEntryList.size(); i++) { + RangeResourceEntry tmp = rangeResourceEntryList.get(i); + if (beginValue >= tmp.getBegin() && endValue <= tmp.getEnd()) { + rangeResourceEntryList.remove(i); + // We already removed the entry. So when beginValue == endValue, + // we dont have to add a new entry + if (tmp.getBegin().equals(tmp.getEnd()) || (tmp.getBegin().equals(beginValue) && tmp.getEnd().equals(endValue))) { + removedResources.add(tmp); + return removedResources; + } + + if (beginValue > tmp.getBegin() && tmp.getEnd().equals(endValue)) { + rangeResourceEntryList.add(new RangeResourceEntry(reservationType, tmp.getBegin(), beginValue - 1)); + removedResources.add(new RangeResourceEntry(reservationType, beginValue, endValue)); + } else if (tmp.getBegin().equals(beginValue) && endValue < tmp.getEnd()) { + rangeResourceEntryList.add(new RangeResourceEntry(reservationType, endValue + 1, tmp.getEnd())); + removedResources.add(new RangeResourceEntry(reservationType, beginValue, endValue)); + } else if (beginValue > tmp.getBegin() && endValue < tmp.getEnd()) { + rangeResourceEntryList.add(new RangeResourceEntry(reservationType, tmp.getBegin(), beginValue - 1)); + rangeResourceEntryList.add(new RangeResourceEntry(reservationType, endValue + 1, tmp.getEnd())); + removedResources.add(new RangeResourceEntry(reservationType, beginValue, endValue)); + } + } + } + } + return removedResources; + } +} diff --git a/storm/src/main/storm/mesos/resources/ReservationType.java b/storm/src/main/storm/mesos/resources/ReservationType.java new file mode 100644 index 000000000..6e0c28390 --- /dev/null +++ b/storm/src/main/storm/mesos/resources/ReservationType.java @@ -0,0 +1,24 @@ +/** + * 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 storm.mesos.resources; + +public enum ReservationType { + UNRESERVED, + STATICALLY_RESERVED, + DYNAMICALLY_RESERVED; +} diff --git a/storm/src/main/storm/mesos/resources/Resource.java b/storm/src/main/storm/mesos/resources/Resource.java new file mode 100644 index 000000000..67346dace --- /dev/null +++ b/storm/src/main/storm/mesos/resources/Resource.java @@ -0,0 +1,45 @@ +/** + * 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 storm.mesos.resources; + +import java.util.Comparator; +import java.util.List; + +/** + * An interface for managing various kinds of resources defined + * in {@link storm.mesos.resources.ResourceType} + */ +public interface Resource> { + + public boolean isAvailable(T resourceEntry); + + public boolean isAvailable(T resourceEntry, ReservationType reservationType); + + public List getAllAvailableResources(); + + public List getAllAvailableResources(ReservationType reservationType); + + public void add(T resourceEntry, ReservationType reservationType); + + public List removeAndGet(T resourceEntry) throws ResourceNotAvailabeException; + + public List removeAndGet(T value, ReservationType reservationType) throws ResourceNotAvailabeException; + + public List removeAndGet(T value, Comparator reservationTypeComparator) throws ResourceNotAvailabeException; + +} diff --git a/storm/src/main/storm/mesos/resources/ResourceEntries.java b/storm/src/main/storm/mesos/resources/ResourceEntries.java new file mode 100644 index 000000000..848ff8edc --- /dev/null +++ b/storm/src/main/storm/mesos/resources/ResourceEntries.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 + * + * 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 storm.mesos.resources; + +public class ResourceEntries { + public static final class ScalarResourceEntry implements ResourceEntry { + private ReservationType reservationType; + private Double value; + + public ReservationType getReservationType() { + return reservationType; + } + + public ScalarResourceEntry(Double value) { + this.value = value; + } + + public ScalarResourceEntry(ReservationType reservationType, Double value) { + this.reservationType = reservationType; + this.value = value; + } + + public Double getValue() { + return value; + } + + @Override + public ScalarResourceEntry add(ResourceEntry resourceEntry) { + ScalarResourceEntry scalarResourceEntry = (ScalarResourceEntry) resourceEntry; + this.value += scalarResourceEntry.getValue(); + return this; + } + + @Override + public ScalarResourceEntry remove(ResourceEntry resourceEntry) { + ScalarResourceEntry scalarResourceEntry = (ScalarResourceEntry) resourceEntry; + this.value -= scalarResourceEntry.getValue(); + return this; + } + } + + public static final class RangeResourceEntry implements ResourceEntry { + private ReservationType reservationType; + private Long begin; + private Long end; + + public RangeResourceEntry(Long begin, Long end) { + this.reservationType = reservationType; + this.begin = begin; + this.end = end; + } + + public RangeResourceEntry(ReservationType reservationType, Long begin, Long end) { + this.reservationType = reservationType; + this.begin = begin; + this.end = end; + } + + public Long getBegin() { + return begin; + } + + public Long getEnd() { + return end; + } + + public ReservationType getReservationType() { + return reservationType; + } + + @Override + public RangeResourceEntry add(ResourceEntry resourceEntry) { + RangeResourceEntry rangeResourceEntry = (RangeResourceEntry) resourceEntry; + if (this.begin < rangeResourceEntry.getBegin()) { + this.begin = rangeResourceEntry.getBegin(); + } + if (this.end > rangeResourceEntry.getBegin()) { + this.end = rangeResourceEntry.getBegin(); + } + return this; + } + + @Override + public RangeResourceEntry remove(ResourceEntry resourceEntry) { + RangeResourceEntry rangeResourceEntry = (RangeResourceEntry) resourceEntry; + if (this.begin > rangeResourceEntry.getBegin()) { + this.begin = rangeResourceEntry.getBegin(); + } + + if (this.end < rangeResourceEntry.getEnd()) { + this.end = rangeResourceEntry.getEnd(); + } + return this; + } + } +} diff --git a/storm/src/main/storm/mesos/resources/ResourceEntry.java b/storm/src/main/storm/mesos/resources/ResourceEntry.java new file mode 100644 index 000000000..d38ebacb9 --- /dev/null +++ b/storm/src/main/storm/mesos/resources/ResourceEntry.java @@ -0,0 +1,31 @@ +/** + * 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 storm.mesos.resources; + +/** + * Interface to represent various resource types defined in + * {@link storm.mesos.resources.ResourceType} + */ +public interface ResourceEntry { + + public ResourceEntry add(ResourceEntry entry); + + public ResourceEntry remove(ResourceEntry entry); + + public ReservationType getReservationType(); +} diff --git a/storm/src/main/storm/mesos/resources/ResourceNotAvailabeException.java b/storm/src/main/storm/mesos/resources/ResourceNotAvailabeException.java new file mode 100644 index 000000000..7d0155237 --- /dev/null +++ b/storm/src/main/storm/mesos/resources/ResourceNotAvailabeException.java @@ -0,0 +1,24 @@ +/** + * 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 storm.mesos.resources; + +public class ResourceNotAvailabeException extends Exception { + public ResourceNotAvailabeException(String message) { + super(message); + } +} diff --git a/storm/src/main/storm/mesos/resources/ResourceType.java b/storm/src/main/storm/mesos/resources/ResourceType.java new file mode 100644 index 000000000..f8a0f4fc3 --- /dev/null +++ b/storm/src/main/storm/mesos/resources/ResourceType.java @@ -0,0 +1,61 @@ +/** + * 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 storm.mesos.resources; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +public enum ResourceType { + CPU("cpus"), + MEM("mem"), + PORTS("ports"), + DISK("disk"); + + private final String resourceType; + private static final Logger LOG = LoggerFactory.getLogger(ResourceType.class); + + private static final Map map = new HashMap<>(ResourceType.values().length); + + static { + for (ResourceType r : ResourceType.values()) { + map.put(r.resourceType, r); + } + } + + + public static ResourceType of(String name) { + ResourceType result = map.get(name); + if (result == null) { + LOG.warn("Unexpected resource type {}", name); + } + return result; + } + + + ResourceType(String resourceType) { + this.resourceType = resourceType; + } + + @Override + public String toString() { + return resourceType; + } +} diff --git a/storm/src/main/storm/mesos/resources/ScalarResource.java b/storm/src/main/storm/mesos/resources/ScalarResource.java new file mode 100644 index 000000000..514b915b6 --- /dev/null +++ b/storm/src/main/storm/mesos/resources/ScalarResource.java @@ -0,0 +1,163 @@ +/** + * 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 storm.mesos.resources; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import static storm.mesos.resources.ResourceEntries.ScalarResourceEntry; + +public class ScalarResource implements Resource { + public final ResourceType resourceType; + private final Map availableResourcesByReservationType; + Double totalAvailableResource = 0.0; + + + public ScalarResource(ResourceType resourceType) { + this.resourceType = resourceType; + availableResourcesByReservationType = new TreeMap<>(new DefaultReservationTypeComparator()); + + for (ReservationType reservationType : ReservationType.values()) { + availableResourcesByReservationType.put(reservationType, new ScalarResourceEntry(reservationType, 0.0)); + } + } + + public boolean isAvailable(ScalarResourceEntry scalarResourceEntry) { + return (totalAvailableResource >= scalarResourceEntry.getValue()); + } + + public boolean isAvailable(ScalarResourceEntry scalarResourceEntry, ReservationType reservationType) { + return (availableResourcesByReservationType.get(reservationType).getValue() <= scalarResourceEntry.getValue()); + } + + public Double getTotalAvailableResource(ReservationType reservationType) { + return availableResourcesByReservationType.get(reservationType).getValue(); + } + + public List getAllAvailableResources() { + List retVal = new ArrayList<>(); + for (ScalarResourceEntry scalarResourceEntry : availableResourcesByReservationType.values()) { + retVal.add(scalarResourceEntry); + } + return retVal; + } + + public List getAllAvailableResources(ReservationType reservationType) { + List retVal = new ArrayList<>(); + retVal.add(availableResourcesByReservationType.get(reservationType)); + return retVal; + } + + public void add(ScalarResourceEntry scalarResourceEntry, ReservationType reservationType) { + if (scalarResourceEntry.getValue() <= 0) { + return; + } + availableResourcesByReservationType.get(reservationType).add(scalarResourceEntry); + totalAvailableResource += scalarResourceEntry.getValue(); + } + + public List removeAndGet(ScalarResourceEntry scalarResourceEntry) throws ResourceNotAvailabeException { + return removeAndGet(scalarResourceEntry, availableResourcesByReservationType.keySet()); + } + + public List reserveScalarResource(ResourceType resourceType, ScalarResourceEntry requiredValue) throws ResourceNotAvailabeException { + if (totalAvailableResource <= requiredValue.getValue()) { + throw new ResourceNotAvailabeException(String.format("resouceType: {} is not available. Requested {} Available {}", + resourceType, requiredValue, totalAvailableResource)); + } + return removeAndGet(requiredValue); + } + + /** + * Removes/Reserves scalar resource from available resources. + * {@param scalarResourceEntry} amount of scalar resource to removeAndGet/decrement + * {@param reservationTypeComparator} comparator like {@link storm.mesos.resources.DefaultReservationTypeComparator} + * to determine the priority of the reservation type. When resources of all reservations are available, resources + * are removed in the priority order specified by this comparator. + */ + public List removeAndGet(ScalarResourceEntry scalarResourceEntry, Comparator reservationTypeComparator) throws ResourceNotAvailabeException { + List reservationTypeList = Arrays.asList(ReservationType.values()); + Collections.sort(reservationTypeList, reservationTypeComparator); + return removeAndGet(scalarResourceEntry, reservationTypeList); + } + + /** + * Removes/Reserves scalar resource from available resources. When resources of all reservations are available, resources + * are removed in the priority order specified by {@link storm.mesos.resources.DefaultReservationTypeComparator} + * {@param scalarResourceEntry} amount of scalar resource to removeAndGet/decrement. + * {@link storm.mesos.resources.DefaultReservationTypeComparator} determines the priority of the reservation type. + */ + public List removeAndGet(ScalarResourceEntry scalarResourceEntry, ReservationType reservationType) throws ResourceNotAvailabeException { + ScalarResourceEntry availableResource = availableResourcesByReservationType.get(reservationType); + List reservedResources = new ArrayList<>(); + + if (scalarResourceEntry.getValue() < availableResource.getValue()) { + availableResourcesByReservationType.put(reservationType, availableResource.remove(scalarResourceEntry)); + totalAvailableResource -= scalarResourceEntry.getValue(); + reservedResources.add(new ScalarResourceEntry(scalarResourceEntry.getReservationType(), scalarResourceEntry.getValue())); + return reservedResources; + } + String message = String.format("ResourceType '%s' of reservationType '%s' is not available. Requested value: %s Available: %s", + resourceType, reservationType.toString(), scalarResourceEntry.getValue(), availableResourcesByReservationType.get(reservationType)); + throw new ResourceNotAvailabeException(message); + } + + public String toString() { + List availableResourcesByResourceTypeList = new ArrayList<>(); + for (Map.Entry entry: availableResourcesByReservationType.entrySet()) { + availableResourcesByResourceTypeList.add(String.format("%s : %s", entry.getKey(), entry.getValue().getValue())); + } + String tmp = String.join(", ", availableResourcesByResourceTypeList); + return String.format("Resouce %s - Total available : %f Total available by reservation type : [ %s ]", resourceType.toString(), totalAvailableResource, tmp); + } + + private List removeAndGet(ScalarResourceEntry scalarResourceEntry, Collection reservationTypesListByPriority) throws ResourceNotAvailabeException { + Double requiredValue = scalarResourceEntry.getValue(); + List reservedResources = new ArrayList<>(); + + if (requiredValue > totalAvailableResource) { + String message = String.format("ResourceType '%s' is not available. Requested value: %s Available: %s", + resourceType, requiredValue, totalAvailableResource); + throw new ResourceNotAvailabeException(message); + } + + for (ReservationType reservationType : reservationTypesListByPriority) { + ScalarResourceEntry availableResource = availableResourcesByReservationType.get(reservationType); + Double availableResourceValue = availableResource.getValue(); + + if (availableResourceValue >= requiredValue) { + availableResource.remove(new ScalarResourceEntry(requiredValue)); + totalAvailableResource -= requiredValue; + reservedResources.add(new ScalarResourceEntry(reservationType, requiredValue)); + return reservedResources; + } else if (availableResourceValue > 0) { + availableResourcesByReservationType.put(reservationType, new ScalarResourceEntry(reservationType, 0.0)); + requiredValue -= availableResourceValue; + totalAvailableResource -= availableResourceValue; + reservedResources.add(new ScalarResourceEntry(reservationType, availableResourceValue)); + } + } + return reservedResources; + } +} diff --git a/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java b/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java index 75adf3e58..a044fc1e8 100644 --- a/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java +++ b/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java @@ -28,6 +28,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import storm.mesos.MesosNimbus; +import storm.mesos.resources.OfferResources; +import storm.mesos.resources.RangeResource; +import storm.mesos.resources.ResourceEntries; +import storm.mesos.resources.ResourceNotAvailabeException; import storm.mesos.util.MesosCommon; import storm.mesos.util.RotatingMap; @@ -45,7 +49,7 @@ * Default Scheduler used by mesos-storm framework. */ public class DefaultScheduler implements IScheduler, IMesosStormScheduler { - private final Logger log = LoggerFactory.getLogger(MesosNimbus.class); + private final Logger log = LoggerFactory.getLogger(DefaultScheduler.class); private Map mesosStormConf; private final Map mesosWorkerSlotMap = new HashMap<>(); @@ -75,33 +79,15 @@ public List allSlotsAvailableForScheduling(RotatingMap existingSupervisors, Topologies topologies, Set topologiesMissingAssignments) { if (topologiesMissingAssignments.isEmpty()) { - log.info("Declining all offers that are currently buffered because no topologies need assignments"); - offers.clear(); + log.info("Not Declining all offers that are currently buffered because no topologies need assignments"); + //offers.clear(); return new ArrayList<>(); } log.info("Topologies that need assignments: {}", topologiesMissingAssignments.toString()); - // Decline those offers that cannot be used for any of the topologies that need assignments. - for (Protos.Offer offer : offers.newestValues()) { - boolean isOfferUseful = false; - for (String currentTopology : topologiesMissingAssignments) { - boolean supervisorExists = SchedulerUtils.supervisorExists(offer.getHostname(), existingSupervisors, currentTopology); - TopologyDetails topologyDetails = topologies.getById(currentTopology); - if (SchedulerUtils.isFit(mesosStormConf, offer, topologyDetails, supervisorExists)) { - isOfferUseful = true; - break; - } - } - if (!isOfferUseful) { - log.info("Declining Offer {} because it does not fit any of the topologies that need assignments", offer.getId().getValue()); - offers.clearKey(offer.getId()); - } - } - List allSlots = new ArrayList<>(); - - Map> offerResourcesListPerNode = SchedulerUtils.getOfferResourcesListPerNode(offers); + Map offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(offers); for (String currentTopology : topologiesMissingAssignments) { TopologyDetails topologyDetails = topologies.getById(currentTopology); @@ -111,50 +97,59 @@ public List allSlotsAvailableForScheduling(RotatingMap nodesWithExistingSupervisors = new HashSet<>(); - for (String currentNode : offerResourcesListPerNode.keySet()) { + for (String currentNode : offerResourcesPerNode.keySet()) { if (SchedulerUtils.supervisorExists(currentNode, existingSupervisors, currentTopology)) { nodesWithExistingSupervisors.add(currentNode); } } - Boolean slotFound; - do { + boolean slotFound = true; + + while (slotsNeeded > 0 && slotFound) { slotFound = false; - for (String currentNode : offerResourcesListPerNode.keySet()) { + for (String currentNode : offerResourcesPerNode.keySet()) { + OfferResources offerResources = offerResourcesPerNode.get(currentNode); + boolean supervisorExists = nodesWithExistingSupervisors.contains(currentNode); - if (slotsNeeded == 0) { - break; + boolean isFit = false; + List availablePorts = SchedulerUtils.getPorts(offerResources, 1); + + if (!availablePorts.isEmpty()) { + isFit = SchedulerUtils.isFit(mesosStormConf, offerResources, topologyDetails, availablePorts.get(0).getBegin(), supervisorExists, MesosCommon.autoStartLogViewer(mesosStormConf)); + } + + if (!isFit) { + log.info("{} is not a fit for {} requestedWorkerCpu: {} requestedWorkerMem: {}", + offerResources.toString(), topologyDetails.getId(), requestedWorkerCpu, requestedWorkerMem); + continue; } - for (OfferResources resources : offerResourcesListPerNode.get(currentNode)) { - boolean isFit = SchedulerUtils.isFit(mesosStormConf, resources, topologyDetails, supervisorExists); - if (isFit) { - log.info("{} is a fit for {} requestedWorkerCpu: {} requestedWorkerMem: {}", resources.toString(), - topologyDetails.getId(), requestedWorkerCpu, requestedWorkerMem); - nodesWithExistingSupervisors.add(currentNode); - MesosWorkerSlot mesosWorkerSlot = SchedulerUtils.createWorkerSlotFromOfferResources(mesosStormConf, resources, topologyDetails, supervisorExists); - if (mesosWorkerSlot == null) { - continue; - } - String slotId = String.format("%s:%s", mesosWorkerSlot.getNodeId(), mesosWorkerSlot.getPort()); - mesosWorkerSlotMap.put(slotId, mesosWorkerSlot); - // Place this offer in the first bucket of the RotatingMap so that it is less likely to get rotated out - offers.put(resources.getOfferId(), resources.getOffer()); - allSlots.add(mesosWorkerSlot); - slotsNeeded--; - slotFound = true; - } else { - log.info("{} is not a fit for {} requestedWorkerCpu: {} requestedWorkerMem: {}", - resources.toString(), topologyDetails.getId(), requestedWorkerCpu, requestedWorkerMem); - } + log.info("{} is a fit for {} requestedWorkerCpu: {} requestedWorkerMem: {}", offerResources.toString(), + topologyDetails.getId(), requestedWorkerCpu, requestedWorkerMem); + nodesWithExistingSupervisors.add(currentNode); + MesosWorkerSlot mesosWorkerSlot; + + try { + mesosWorkerSlot = SchedulerUtils.createMesosWorkerSlot(mesosStormConf, offerResources, topologyDetails, supervisorExists); + } catch (ResourceNotAvailabeException rexp) { + log.warn(rexp.getMessage()); + continue; + } + + String slotId = String.format("%s:%s", mesosWorkerSlot.getNodeId(), mesosWorkerSlot.getPort()); + mesosWorkerSlotMap.put(slotId, mesosWorkerSlot); + allSlots.add(mesosWorkerSlot); + slotFound = true; + if (--slotsNeeded == 0) { + break; } } - } while (slotFound == true && slotsNeeded > 0); + } log.info("Number of available slots for {} : {}", topologyDetails.getId(), allSlots.size()); } diff --git a/storm/src/main/storm/mesos/schedulers/OfferResources.java b/storm/src/main/storm/mesos/schedulers/OfferResources.java deleted file mode 100644 index beac0e1a6..000000000 --- a/storm/src/main/storm/mesos/schedulers/OfferResources.java +++ /dev/null @@ -1,156 +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 storm.mesos.schedulers; - -import com.google.common.base.Joiner; -import org.slf4j.LoggerFactory; -import org.slf4j.Logger; -import org.apache.mesos.Protos; - -import java.util.ArrayList; -import java.util.List; - -public class OfferResources { - private final Logger log = LoggerFactory.getLogger(OfferResources.class); - - private class PortRange { - public long begin; - public long end; - - public PortRange(long begin, long end) { - this.begin = begin; - this.end = end; - } - } - - private Protos.Offer offer; - private Protos.OfferID offerId; - private String hostName; - private double mem; - private double cpu; - - List portRanges = new ArrayList<>(); - - public void addPortRanges(Protos.Value.Ranges ranges) { - for (Protos.Value.Range r : ranges.getRangeList()) { - this.portRanges.add(new PortRange(r.getBegin(), r.getEnd())); - } - } - - public OfferResources(Protos.Offer offer) { - this.offer = offer; - this.offerId = offer.getId(); - double offerMem = 0; - double offerCpu = 0; - Protos.Value.Ranges portRanges = null; - - String hostName = offer.getHostname(); - for (Protos.Resource r : offer.getResourcesList()) { - if (r.hasReservation()) { - // skip resources with dynamic reservations - continue; - } - if (r.getName().equals("cpus")) { - offerCpu += r.getScalar().getValue(); - } else if (r.getName().equals("mem")) { - offerMem += r.getScalar().getValue(); - } else if (r.getName().equals("ports")) { - Protos.Value.Ranges tmp = r.getRanges(); - if (portRanges == null) { - portRanges = tmp; - continue; - } - portRanges.getRangeList().addAll(tmp.getRangeList()); - } - } - - this.hostName = hostName; - this.mem = offerMem; - this.cpu = offerCpu; - if ((portRanges != null) && (!portRanges.getRangeList().isEmpty())) { - this.addPortRanges(portRanges); - } - } - - public Protos.Offer getOffer() { - return this.offer; - } - - public Protos.OfferID getOfferId() { - return this.offerId; - } - - public String getHostName() { - return this.hostName; - } - - public double getMem() { - return this.mem; - } - - public double getCpu() { - return this.cpu; - } - - public void decCpu(double val) { - cpu -= val; - } - - public void decMem(double val) { - mem -= val; - } - - public long getPort() { - if (!hasPort()) { - return -1; - } - - for (int i = 0; i < portRanges.size(); i++) { - PortRange portRange = portRanges.get(i); - if (portRange.begin < portRange.end) { - return portRange.begin++; - } else if (portRange.begin == portRange.end) { - portRanges.remove(i); - return portRange.begin; - } - } - - return -1; - } - - public boolean hasPort() { - return (portRanges != null && !portRanges.isEmpty()); - } - - @Override - public String toString() { - List portRangeStrings = new ArrayList<>(); - - for (int i = 0; i < portRanges.size(); i++) { - if (portRanges.get(i).begin == portRanges.get(i).end) { - portRangeStrings.add(String.format("%d", portRanges.get(i).begin)); - } else { - portRangeStrings.add(String.format("%d-%d", portRanges.get(i).begin, portRanges.get(i).end)); - } - } - // TODO(erikdw): limit float's resolution? - return String.format("OfferResources with offerId: %s from host: %s mem: %f cpu: %f portRanges: [%s]", - getOfferId().getValue(), getHostName(), mem, cpu, Joiner.on(",").join(portRangeStrings)); - } -} - diff --git a/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java b/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java index 2f938fad8..a7b55f975 100644 --- a/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java +++ b/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java @@ -22,6 +22,11 @@ import org.apache.mesos.Protos; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import storm.mesos.resources.OfferResources; +import storm.mesos.resources.RangeResource; +import storm.mesos.resources.ResourceEntry; +import storm.mesos.resources.ResourceNotAvailabeException; +import storm.mesos.resources.ResourceType; import storm.mesos.util.MesosCommon; import storm.mesos.util.RotatingMap; @@ -31,26 +36,66 @@ import java.util.List; import java.util.Map; +import static storm.mesos.resources.ResourceEntries.*; + public class SchedulerUtils { private static final Logger log = LoggerFactory.getLogger(SchedulerUtils.class); - public static boolean isFit(Map mesosStormConf, OfferResources offerResources, TopologyDetails topologyDetails, boolean supervisorExists) { + public static List getPorts(OfferResources offerResources, int requiredCount) { + List retVal = new ArrayList<>(); + List resourceEntryList = offerResources.getAllAvailableResources(ResourceType.PORTS); + + for (RangeResourceEntry rangeResourceEntry : resourceEntryList) { + Long begin = rangeResourceEntry.getBegin(); + Long end = rangeResourceEntry.getEnd(); + for (int i = 0; i <= (end - begin) && requiredCount > 0; i++) { + retVal.add(new RangeResourceEntry(begin, begin + i)); + requiredCount--; + } + } + + return retVal; + } + + public static boolean isFit(Map mesosStormConf, + OfferResources offerResources, + TopologyDetails topologyDetails, + Long port, + boolean supervisorExists, + boolean autoStartLogviewer) { + double requestedWorkerCpu = MesosCommon.topologyWorkerCpu(mesosStormConf, topologyDetails); double requestedWorkerMem = MesosCommon.topologyWorkerMem(mesosStormConf, topologyDetails); requestedWorkerCpu += supervisorExists ? 0 : MesosCommon.executorCpu(mesosStormConf); requestedWorkerMem += supervisorExists ? 0 : MesosCommon.executorMem(mesosStormConf); - if (requestedWorkerCpu <= offerResources.getCpu() && requestedWorkerMem <= offerResources.getMem()) { - return true; - } - return false; + return (offerResources.isAvaliable(ResourceType.CPU, new ScalarResourceEntry(requestedWorkerCpu)) && + offerResources.isAvaliable(ResourceType.MEM, new ScalarResourceEntry(requestedWorkerMem)) && + offerResources.isAvaliable(ResourceType.PORTS, new RangeResourceEntry(port, port))); } - public static boolean isFit(Map mesosStormConf, Protos.Offer offer, TopologyDetails topologyDetails, boolean supervisorExists) { - OfferResources offerResources = new OfferResources(offer); - return isFit(mesosStormConf, offerResources, topologyDetails, supervisorExists); + + + public static MesosWorkerSlot createMesosWorkerSlot(Map mesosStormConf, + OfferResources offerResources, + TopologyDetails topologyDetails, + boolean supervisorExists) throws ResourceNotAvailabeException { + + double requestedWorkerCpu = MesosCommon.topologyWorkerCpu(mesosStormConf, topologyDetails); + double requestedWorkerMem = MesosCommon.topologyWorkerMem(mesosStormConf, topologyDetails); + + requestedWorkerCpu += supervisorExists ? 0 : MesosCommon.executorCpu(mesosStormConf); + requestedWorkerMem += supervisorExists ? 0 : MesosCommon.executorMem(mesosStormConf); + + offerResources.reserve(ResourceType.CPU, new ScalarResourceEntry(requestedWorkerCpu)); + offerResources.reserve(ResourceType.MEM, new ScalarResourceEntry(requestedWorkerMem)); + + List ports = getPorts(offerResources, 1); + offerResources.reserve(ResourceType.PORTS, ports.get(0)); + + return new MesosWorkerSlot(offerResources.getHostName(), ports.get(0).getBegin(), topologyDetails.getId()); } /** @@ -93,36 +138,5 @@ public static Map> getOfferResourcesListPerNode(Rot return offerResourcesListPerNode; } - public static MesosWorkerSlot createWorkerSlotFromOfferResources(Map mesosStormConf, OfferResources offerResources, - TopologyDetails topologyDetails, boolean supervisorExists) { - double requestedWorkerCpu = MesosCommon.topologyWorkerCpu(mesosStormConf, topologyDetails); - double requestedWorkerMem = MesosCommon.topologyWorkerMem(mesosStormConf, topologyDetails); - requestedWorkerCpu += supervisorExists ? 0 : MesosCommon.executorCpu(mesosStormConf); - requestedWorkerMem += supervisorExists ? 0 : MesosCommon.executorMem(mesosStormConf); - - if (requestedWorkerCpu > offerResources.getCpu()) { - log.warn("Refusing to create worker slot. requestedWorkerCpu: {} but OfferedCpu: {} at node: {}", - requestedWorkerCpu, offerResources.getCpu(), offerResources.getHostName()); - return null; - } - - if (requestedWorkerMem > offerResources.getMem()) { - log.warn("Refusing to create worker slot. requestedWorkerMem: {} but OfferedMem: {} at node: {}", - requestedWorkerMem, offerResources.getMem(), offerResources.getHostName()); - return null; - } - - long port = offerResources.getPort(); - - if (port == -1) { - log.warn("Refusing to create worker slot. There are no ports available with offer {}", offerResources.toString()); - return null; - } - - offerResources.decCpu(requestedWorkerCpu); - offerResources.decMem(requestedWorkerMem); - - return new MesosWorkerSlot(offerResources.getHostName(), port, topologyDetails.getId()); - } } diff --git a/storm/src/main/storm/mesos/util/MesosCommon.java b/storm/src/main/storm/mesos/util/MesosCommon.java index 47b99fc7a..5095fcc5d 100644 --- a/storm/src/main/storm/mesos/util/MesosCommon.java +++ b/storm/src/main/storm/mesos/util/MesosCommon.java @@ -17,11 +17,16 @@ */ package storm.mesos.util; +import backtype.storm.Config; import backtype.storm.scheduler.TopologyDetails; import com.google.common.base.Optional; +import org.apache.mesos.Protos; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import storm.mesos.resources.OfferResources; +import java.net.InetAddress; +import java.net.UnknownHostException; import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -35,6 +40,7 @@ public class MesosCommon { public static final String EXECUTOR_MEM_CONF = "topology.mesos.executor.mem.mb"; public static final String SUICIDE_CONF = "mesos.supervisor.suicide.inactive.timeout.secs"; public static final String SUPERVISOR_STORM_LOCAL_DIR_CONF = "mesos.supervisor.storm.local.dir"; + public static final String CONF_MESOS_ROLE = "mesos.framework.role"; public static final String AUTO_START_LOGVIEWER_CONF = "supervisor.autostart.logviewer"; // Should we prefix the Worker Task ID with a configurable string (as well as the topology name)? public static final String WORKER_NAME_PREFIX = "topology.mesos.worker.prefix"; @@ -55,6 +61,14 @@ public class MesosCommon { public static final String DEFAULT_WORKER_NAME_PREFIX_DELIMITER = "_"; public static final String DEFAULT_MESOS_COMPONENT_NAME_DELIMITER = " | "; + + public static String getNimbusHost(Map mesosStormConf) throws UnknownHostException { + Optional nimbusHostFromConfig = Optional.fromNullable((String) mesosStormConf.get(Config.NIMBUS_HOST)); + Optional nimbusHostFromEnv = Optional.fromNullable(System.getenv("MESOS_NIMBUS_HOST")); + + return nimbusHostFromConfig.or(nimbusHostFromEnv).or(nimbusHostFromConfig).or(InetAddress.getLocalHost().getCanonicalHostName()); + } + public static String hostFromAssignmentId(String assignmentId, String delimiter) { final int last = assignmentId.lastIndexOf(delimiter); String host = assignmentId.substring(last + delimiter.length()); @@ -94,7 +108,7 @@ public static String supervisorId(String nodeid, String topologyId) { return String.format("%s-%s", nodeid, topologyId); } - public static boolean startLogViewer(Map conf) { + public static boolean autoStartLogViewer(Map conf) { return Optional.fromNullable((Boolean) conf.get(AUTO_START_LOGVIEWER_CONF)).or(true); } @@ -125,6 +139,24 @@ public static int portFromTaskId(String taskId) { return port; } + public static Map getConsolidatedOfferResourcesPerNode(RotatingMap offers) { + Map offerResourcesPerNode = new HashMap<>(); + + for (Protos.Offer offer : offers.values()) { + String hostName = offer.getHostname(); + + OfferResources offerResourcesForHost = offerResourcesPerNode.get(hostName); + if (offerResourcesForHost == null) { + offerResourcesForHost = new OfferResources(offer); + offerResourcesPerNode.put(hostName, offerResourcesForHost); + } else { + offerResourcesForHost.add(offer); + } + LOG.info("Available resources at " + hostName + ": " + offerResourcesForHost.toString()); + } + return offerResourcesPerNode; + } + public static int getSuicideTimeout(Map conf) { return Optional.fromNullable((Number) conf.get(SUICIDE_CONF)) .or(DEFAULT_SUICIDE_TIMEOUT_SECS).intValue(); @@ -184,4 +216,9 @@ public static double executorMem(Map conf) { return Optional.fromNullable((Number) conf.get(EXECUTOR_MEM_CONF)) .or(DEFAULT_EXECUTOR_MEM_MB).doubleValue(); } + + public static String getRole(Map conf) { + return Optional.fromNullable((String) conf.get(CONF_MESOS_ROLE)).or("*"); + } + } diff --git a/storm/src/main/storm/mesos/util/PrettyProtobuf.java b/storm/src/main/storm/mesos/util/PrettyProtobuf.java index b312c49d8..ebf3c4efa 100644 --- a/storm/src/main/storm/mesos/util/PrettyProtobuf.java +++ b/storm/src/main/storm/mesos/util/PrettyProtobuf.java @@ -230,4 +230,19 @@ private static Map resourcesToOrderedMap(List resource } return map; } + + /** + * Wrapper around getTrimmedString which allows using gauva's transform utility. + */ + private static Function offerIDToStringTransform = + new Function() { + public String apply(OfferID o) { + return o.getValue().toString(); + } + }; + + public static String offerIDListToString(List offerIDList) { + List offerIDsAsStrings = Lists.transform(offerIDList, offerIDToStringTransform); + return "[" + StringUtils.join(offerIDsAsStrings, ", ") + "]"; + } } diff --git a/storm/src/main/storm/mesos/util/RotatingMap.java b/storm/src/main/storm/mesos/util/RotatingMap.java index 6d6a6ace3..399e29bc3 100644 --- a/storm/src/main/storm/mesos/util/RotatingMap.java +++ b/storm/src/main/storm/mesos/util/RotatingMap.java @@ -31,7 +31,7 @@ * The algorithm used will take between expirationSecs and * expirationSecs * (1 + 1 / (numBuckets-1)) to actually expire the message. *

- * get, put, remove, containsKey, and size take O(numBuckets) time to run. + * get, put, removeAndGet, containsKey, and size take O(numBuckets) time to run. *

* The advantage of this design is that the expiration thread only locks the object * for O(1) time, meaning the object is essentially always available for gets/puts. diff --git a/storm/src/test/storm/mesos/MesosCommonTest.java b/storm/src/test/storm/mesos/MesosCommonTest.java index 1b11edbf6..645b5900f 100644 --- a/storm/src/test/storm/mesos/MesosCommonTest.java +++ b/storm/src/test/storm/mesos/MesosCommonTest.java @@ -19,9 +19,14 @@ import backtype.storm.generated.StormTopology; import backtype.storm.scheduler.TopologyDetails; +import org.apache.mesos.Protos; import org.junit.Before; import org.junit.Test; +import storm.mesos.resources.OfferResources; +import storm.mesos.resources.ReservationType; +import storm.mesos.resources.ResourceType; import storm.mesos.util.MesosCommon; +import storm.mesos.util.RotatingMap; import java.util.HashMap; import java.util.Map; @@ -35,7 +40,13 @@ public class MesosCommonTest { private TopologyDetails info; private String topologyName = "t_name"; private static final double DELTA = 0.0001; + private final MesosNimbus mesosNimbus; + public MesosCommonTest() { + Map mesosStromConfig = new HashMap(); + mesosNimbus = new MesosNimbus(); + mesosNimbus.initializeMesosStormConf(mesosStromConfig, "/mock"); + } @Before public void initConf() { @@ -150,10 +161,10 @@ public void testSupervisorId() throws Exception { @Test public void testStartLogViewer() throws Exception { // Test the default (true) - boolean result = MesosCommon.startLogViewer(conf); + boolean result = MesosCommon.autoStartLogViewer(conf); assertTrue(result); conf.put(MesosCommon.AUTO_START_LOGVIEWER_CONF, false); - result = MesosCommon.startLogViewer(conf); + result = MesosCommon.autoStartLogViewer(conf); assertTrue(!result); } @@ -317,4 +328,32 @@ public void testExecutorMem() throws Exception { expectedResult = 100; assertEquals(result, expectedResult, DELTA); } + + @Test + public void getConsolidatedOfferResourcesPerNode() { + RotatingMap r = new RotatingMap(2); + Protos.Offer offer = TestUtils.buildOffer("0-1", "h1", 0, 0); + r.put(offer.getId(), offer); + offer = TestUtils.buildOffer("0-2", "h1", 10, 1000); + r.put(offer.getId(), offer); + offer = TestUtils.buildOfferWithReservation("0-1", "h1", 0, 0, 200, 2000); + r.put(offer.getId(), offer); + + offer = TestUtils.buildOffer("0-3", "h2", 0, 0); + r.put(offer.getId(), offer); + offer = TestUtils.buildOfferWithPorts("O-4", "h2", 0, 0, 1, 100); + r.put(offer.getId(), offer); + + Map offerResourcesMap = MesosCommon.getConsolidatedOfferResourcesPerNode(r); + OfferResources offerResources = offerResourcesMap.get("h1"); + assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU) == 210); + assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM) == 3000); + assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU, ReservationType.STATICALLY_RESERVED) == 200); + assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM, ReservationType.STATICALLY_RESERVED) == 2000); + assertTrue(TestUtils.calculateAllAvailableRangeResources(offerResources, ResourceType.PORTS, ReservationType.STATICALLY_RESERVED).size() == 0); + offerResources = offerResourcesMap.get("h2"); + assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU, ReservationType.STATICALLY_RESERVED) == 0); + assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM, ReservationType.STATICALLY_RESERVED) == 0); + assertTrue(TestUtils.calculateAllAvailableRangeResources(offerResources, ResourceType.PORTS, ReservationType.UNRESERVED).size() == 100); + } } diff --git a/storm/src/test/storm/mesos/MesosNimbusTest.java b/storm/src/test/storm/mesos/MesosNimbusTest.java index 6e8388ac8..fde746a46 100644 --- a/storm/src/test/storm/mesos/MesosNimbusTest.java +++ b/storm/src/test/storm/mesos/MesosNimbusTest.java @@ -17,304 +17,451 @@ */ package storm.mesos; -import backtype.storm.generated.StormTopology; import backtype.storm.scheduler.Topologies; import backtype.storm.scheduler.TopologyDetails; import backtype.storm.scheduler.WorkerSlot; +import org.apache.mesos.Protos; import org.apache.mesos.Protos.Offer; import org.apache.mesos.Protos.OfferID; +import org.junit.Before; import org.junit.Test; +import org.mockito.Mockito; +import storm.mesos.resources.OfferResources; +import storm.mesos.resources.ReservationType; +import storm.mesos.resources.Resource; +import storm.mesos.resources.ResourceType; import storm.mesos.util.MesosCommon; import storm.mesos.util.RotatingMap; -import java.net.URI; -import java.util.Arrays; +import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; // TODO(dskarthick) : Leverage the build methods defined in TestUtils function. public class MesosNimbusTest { - @Test - public void testGetResourcesScalar() throws Exception { - MesosNimbus mesosNimbus = new MesosNimbus(); - - assertEquals( - Arrays.asList(TestUtils.buildScalarResource("cpus", 1.0)), - mesosNimbus.getResourcesScalar( - TestUtils.buildResourceList(1, 2, 3, 4), - 1.0, - "cpus" - ) - ); - - assertEquals( - Arrays.asList(TestUtils.buildScalarResource("mem", 2.0)), - mesosNimbus.getResourcesScalar( - TestUtils.buildResourceList(1, 2, 3, 4), - 2.0, - "mem" - ) - ); - - assertEquals( - Arrays.asList( - TestUtils.buildScalarResource("cpus", 1.0), - TestUtils.buildScalarResourceWithRole("cpus", 1.0, "reserved") - ), - mesosNimbus.getResourcesScalar( - TestUtils.buildResourceList(1, 2, 3, 4), - 2.0, - "cpus" - ) - ); - - assertEquals( - Arrays.asList( - TestUtils.buildScalarResource("mem", 2.0), - TestUtils.buildScalarResourceWithRole("mem", 1.0, "reserved") - ), - mesosNimbus.getResourcesScalar( - TestUtils.buildResourceList(1, 2, 3, 4), - 3.0, - "mem" - ) - ); - - assertEquals( - Arrays.asList( - TestUtils.buildScalarResource("cpus", 1.0), - TestUtils.buildScalarResourceWithRole("cpus", 3.0, "reserved") - ), - mesosNimbus.getResourcesScalar( - TestUtils.buildResourceList(1, 2, 3, 4), - 4.0, - "cpus" - ) - ); - - assertEquals( - Arrays.asList( - TestUtils.buildScalarResource("mem", 2.0), - TestUtils.buildScalarResourceWithRole("mem", 4.0, "reserved") - ), - mesosNimbus.getResourcesScalar( - TestUtils.buildResourceList(1, 2, 3, 4), - 6.0, - "mem" - ) - ); - - assertEquals( - Arrays.asList( - TestUtils.buildScalarResource("cpus", 1.0), - TestUtils.buildScalarResourceWithRole("cpus", 0.5, "reserved") - ), - mesosNimbus.getResourcesScalar( - TestUtils.buildResourceList(1, 2, 3, 4), - 1.5, - "cpus" - ) - ); - - assertEquals( - Arrays.asList( - TestUtils.buildScalarResource("mem", 2.0), - TestUtils.buildScalarResourceWithRole("mem", 0.5, "reserved") - ), - mesosNimbus.getResourcesScalar( - TestUtils.buildResourceList(1, 2, 3, 4), - 2.5, - "mem" - ) - ); + public static final Integer DEFAULT_WORKER_COUNT = 2; + public static final Integer DEFAULT_BUCKET_COUNT = 2; + public static final String FRAMEWORK_ROLE = "staas"; + + private RotatingMap rotatingMap = null; + Map> slotsForTopologiesNeedingAssignments = null; + MesosNimbus mesosNimbus = null; + Map mesosStormConf; + + private boolean hasResources(String role, List resourceList, Double cpus, Double mem, Long port) { + Double actualCpu = 0.0d, actualMem = 0.0d; + List expectedPorts = new ArrayList<>(); + expectedPorts.add(port); + + List actualPorts = new ArrayList<>(); + for (Protos.Resource resource : resourceList) { + if (!resource.getRole().equals(role)) { + continue; + } + ResourceType r = ResourceType.of(resource.getName()); + switch (r) { + case CPU: + actualCpu += resource.getScalar().getValue(); + break; + case MEM: + actualMem += resource.getScalar().getValue(); + break; + case PORTS: + Protos.Value.Ranges ranges = resource.getRanges(); + for (Protos.Value.Range range : ranges.getRangeList()) { + for (long i = 0; i < (range.getEnd() - range.getBegin() + 1); i++) { + actualPorts.add(range.getBegin() + i); + } + } + } + } + + boolean hasPorts = (expectedPorts.size() == actualPorts.size()) && expectedPorts.containsAll(actualPorts); + return actualCpu.equals(cpus) && actualMem.equals(mem) && hasPorts; } - @Test - public void testSubtractResourcesScalar() throws Exception { - MesosNimbus mesosNimbus = new MesosNimbus(); - - assertEquals( - Arrays.asList( - TestUtils.buildScalarResource("mem", 2.0), - TestUtils.buildScalarResourceWithRole("cpus", 3.0, "reserved"), - TestUtils.buildScalarResourceWithRole("mem", 4.0, "reserved") - ), - mesosNimbus.subtractResourcesScalar( - TestUtils.buildResourceList(1, 2, 3, 4), - 1.0, - "cpus" - ) - ); - - assertEquals( - Arrays.asList( - TestUtils.buildScalarResource("cpus", 1.0), - TestUtils.buildScalarResource("mem", 1.0), - TestUtils.buildScalarResourceWithRole("cpus", 3.0, "reserved"), - TestUtils.buildScalarResourceWithRole("mem", 4.0, "reserved") - ), - mesosNimbus.subtractResourcesScalar( - TestUtils.buildResourceList(1, 2, 3, 4), - 1.0, - "mem" - ) - ); - - assertEquals( - Arrays.asList( - TestUtils.buildScalarResource("mem", 2.0), - TestUtils.buildScalarResourceWithRole("cpus", 2.5, "reserved"), - TestUtils.buildScalarResourceWithRole("mem", 4.0, "reserved") - ), - mesosNimbus.subtractResourcesScalar( - TestUtils.buildResourceList(1, 2, 3, 4), - 1.5, - "cpus" - ) - ); - - assertEquals( - Arrays.asList( - TestUtils.buildScalarResource("cpus", 1.0), - TestUtils.buildScalarResourceWithRole("cpus", 3.0, "reserved"), - TestUtils.buildScalarResourceWithRole("mem", 3.5, "reserved") - ), - mesosNimbus.subtractResourcesScalar( - TestUtils.buildResourceList(1, 2, 3, 4), - 2.5, - "mem" - ) - ); + private boolean hasResources(List resourceList, Double cpus, Double mem, Long port) { + Double actualCpu = 0.0d, actualMem = 0.0d; + List expectedPorts = new ArrayList<>(); + expectedPorts.add(port); + + List actualPorts = new ArrayList<>(); + for (Protos.Resource resource : resourceList) { + ResourceType r = ResourceType.of(resource.getName()); + switch (r) { + case CPU: + actualCpu += resource.getScalar().getValue(); + break; + case MEM: + actualMem += resource.getScalar().getValue(); + break; + case PORTS: + Protos.Value.Ranges ranges = resource.getRanges(); + for (Protos.Value.Range range : ranges.getRangeList()) { + for (long i = 0; i < (range.getEnd() - range.getBegin() + 1); i++) { + actualPorts.add(range.getBegin() + i); + } + } + } + } + + boolean hasPorts = (expectedPorts.size() == actualPorts.size()) && expectedPorts.containsAll(actualPorts); + return actualCpu.equals(cpus) && actualMem.equals(mem) && hasPorts; } - @Test - public void testGetResourcesRange() throws Exception { - MesosNimbus mesosNimbus = new MesosNimbus(); - - assertEquals( - Arrays.asList( - TestUtils.buildRangeResource("ports", 100, 100) - ), - mesosNimbus.getResourcesRange( - TestUtils.buildRangeResourceList(100, 100), - 100, - "ports" - ) - ); - - assertEquals( - Arrays.asList( - TestUtils.buildRangeResource("ports", 105, 105) - ), - mesosNimbus.getResourcesRange( - TestUtils.buildRangeResourceList(100, 200), - 105, - "ports" - ) - ); - - assertEquals( - 0, - mesosNimbus.getResourcesRange( - TestUtils.buildRangeResourceList(100, 100), - 200, - "ports" - ).size() - ); + private boolean hasResources(List resourceList, Double cpus, Double mem) { + Double actualCpu = 0.0d, actualMem = 0.0d; + + for (Protos.Resource resource : resourceList) { + ResourceType r = ResourceType.of(resource.getName()); + switch (r) { + case CPU: + actualCpu += resource.getScalar().getValue(); + break; + case MEM: + actualMem += resource.getScalar().getValue(); + break; + } + } + + return actualCpu.equals(cpus) && actualMem.equals(mem); } + private boolean hasResources(String role, Protos.TaskInfo taskInfo, Double cpus, Double mem) { + Double actualCpu = 0.0d, actualMem = 0.0d; + + for (Protos.Resource resource : taskInfo.getResourcesList()) { + ResourceType r = ResourceType.of(resource.getName()); + if (!resource.getRole().equals(role)) { + continue; + } + switch (r) { + case CPU: + actualCpu += resource.getScalar().getValue(); + break; + case MEM: + actualMem += resource.getScalar().getValue(); + break; + } + } + + return actualCpu.equals(cpus) && actualMem.equals(mem); + } - @Test - public void testSubtractResourcesRange() throws Exception { - MesosNimbus mesosNimbus = new MesosNimbus(); - - assertEquals( - Arrays.asList( - TestUtils.buildScalarResource("cpus", 1.0), - TestUtils.buildScalarResource("mem", 2.0), - TestUtils.buildScalarResourceWithRole("cpus", 3.0, "reserved"), - TestUtils.buildScalarResourceWithRole("mem", 4.0, "reserved") - ), - mesosNimbus.subtractResourcesRange( - TestUtils.buildRangeResourceList(100, 100), - 100, - "ports" - ) - ); - - assertEquals( - Arrays.asList( - TestUtils.buildMultiRangeResource("ports", 100, 104, 106, 200), - TestUtils.buildMultiRangeResourceWithRole("ports", 100, 104, 106, 200, "reserved"), - TestUtils.buildScalarResource("cpus", 1.0), - TestUtils.buildScalarResource("mem", 2.0), - TestUtils.buildScalarResourceWithRole("cpus", 3.0, "reserved"), - TestUtils.buildScalarResourceWithRole("mem", 4.0, "reserved") - ), - mesosNimbus.subtractResourcesRange( - TestUtils.buildRangeResourceList(100, 200), - 105, - "ports" - ) - ); + private boolean hasResources(Protos.TaskInfo taskInfo, Double cpus, Double mem, Long port) { + return hasResources(taskInfo.getResourcesList(), cpus, mem, port); + } + + private boolean hasResources(Protos.TaskInfo taskInfo, Double cpus, Double mem) { + return hasResources(taskInfo.getResourcesList(), cpus, mem); + } + + private boolean hasResources(String role, Protos.TaskInfo taskInfo, Double cpus, Double mem, Long port) { + return hasResources(role, taskInfo.getResourcesList(), cpus, mem, port); + } + + private boolean hasCorrectExecutorResources(List taskInfoList) { + for (Protos.TaskInfo taskInfo : taskInfoList) { + if (!hasResources(taskInfo.getExecutor().getResourcesList(), MesosCommon.DEFAULT_EXECUTOR_CPU, MesosCommon.DEFAULT_EXECUTOR_MEM_MB)) { + return false; + } + } + return true; } + private String getTopologyIdFromTaskName(String taskName) { + String info[] = taskName.split("\\|"); + return info[1]; + } + + private Map> getTopologyIDtoTaskInfoMap(List taskInfoList) { + Map> topologyIDtoTaskInfoMap = new HashMap<>(); + + for (Protos.TaskInfo taskInfo: taskInfoList) { + String topologyId = getTopologyIdFromTaskName(taskInfo.getName()).trim(); + if (!topologyIDtoTaskInfoMap.containsKey(topologyId)) { + topologyIDtoTaskInfoMap.put(topologyId, new ArrayList()); + } + topologyIDtoTaskInfoMap.get(topologyId).add(taskInfo); + } + return topologyIDtoTaskInfoMap; + } + + @Before + public void initialize() { + rotatingMap = new RotatingMap<>(DEFAULT_BUCKET_COUNT); + slotsForTopologiesNeedingAssignments = new HashMap<>(); + + mesosStormConf = new HashMap<>(); + mesosStormConf.put(MesosNimbus.CONF_EXECUTOR_URI, "/fake/path/to/storm-mesos.tgz"); + mesosStormConf.put(MesosCommon.CONF_MESOS_ROLE, FRAMEWORK_ROLE); + mesosNimbus = Mockito.spy(new MesosNimbus()); + + mesosNimbus.initializeMesosStormConf(mesosStormConf, "/mock"); + Mockito.doReturn("http://127.0.0.1/").when(mesosNimbus).getFullConfigUri(); + } @Test - public void testComputeResourcesForSlot() throws Exception { - MesosNimbus mesosNimbus = new MesosNimbus(); + public void testGetTasksToLaunchWhenNoTopologiesNeedAssignments() { + TopologyDetails t1 = TestUtils.constructTopologyDetails("t1", + MesosNimbusTest.DEFAULT_WORKER_COUNT, + MesosCommon.DEFAULT_WORKER_CPU, + MesosCommon.DEFAULT_WORKER_MEM_MB); + Map topologyDetailsMap = new HashMap<>(); + topologyDetailsMap.put("t1", t1); - mesosNimbus._configUrl = new URI("http://127.0.0.1/"); + Topologies topologies = new Topologies(topologyDetailsMap); - OfferID offerId = OfferID.newBuilder().setValue("derp").build(); - RotatingMap offers = new RotatingMap<>( - new RotatingMap.ExpiredCallback() { - @Override - public void expire(OfferID key, Offer val) { - } - } - ); - - offers.put( - offerId, - TestUtils.buildOfferWithPorts("offer1", "host1.west", 2.0, 2048, 1000, 1000) - ); - - HashMap topologyMap = new HashMap<>(); - Map conf = new HashMap<>(); - conf.put(MesosCommon.WORKER_CPU_CONF, 1); - conf.put(MesosCommon.WORKER_MEM_CONF, 1024); - conf.put(MesosCommon.EXECUTOR_CPU_CONF, 1); - conf.put(MesosCommon.EXECUTOR_MEM_CONF, 1024); - conf.put(MesosNimbus.CONF_EXECUTOR_URI, ""); - mesosNimbus._conf = conf; - - topologyMap.put("t1", new TopologyDetails("t1", conf, new StormTopology(), 5)); - HashMap> launchList = new HashMap<>(); - HashMap> slotList = new HashMap<>(); - slotList.put(offerId, Arrays.asList(new WorkerSlot("", 1000))); - Topologies topologies = new Topologies(topologyMap); - - mesosNimbus.computeResourcesForSlot( - offers, - topologies, - launchList, - "t1", - slotList, - OfferID.newBuilder().setValue("derp").build() - ); - assertEquals(1, launchList.size()); - assertEquals(1, launchList.get(offerId).size()); - - assertEquals( - TestUtils.buildScalarResource("cpus", 1.0), - launchList.get(offerId).get(0).getTask().getResources(0) - ); - - assertEquals(0, offers.get(offerId).getResourcesCount()); + Map offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + Map> workerSlotsMap = new HashMap<>(); + + Map> tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + + assertTrue(tasksToLaunch.isEmpty()); + } + + @Test + public void testGetTasksToLaunchForOneTopologyWithOneOffer() { + TopologyDetails t1 = TestUtils.constructTopologyDetails("t1", + MesosNimbusTest.DEFAULT_WORKER_COUNT, + MesosCommon.DEFAULT_WORKER_CPU, + MesosCommon.DEFAULT_WORKER_MEM_MB); + Map m = t1.getConf(); + Map topologyDetailsMap = new HashMap<>(); + topologyDetailsMap.put("t1", t1); + + Topologies topologies = new Topologies(topologyDetailsMap); + + // One offer with sufficient resources + Offer offer = TestUtils.buildOfferWithPorts("O-1", "h1", 24, 40000, 3100, 3200); + rotatingMap.put(offer.getId(), offer); + + Map offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + Map> workerSlotsMap = new HashMap<>(); + Collection workerSlots = new ArrayList<>(); + workerSlots.add(new WorkerSlot("h1", 3100)); + workerSlotsMap.put("t1", workerSlots); + + Map> tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + + assertTrue((tasksToLaunch.size() == 1) && (tasksToLaunch.get("h1").size() == 1)); + + // One offer with sufficient resources spread across reserved and unreserved resources + offer = TestUtils.buildOfferWithReservationAndPorts("O-1", "h1", 0.75, 750, 0.75, 850, 3100, 3101); + rotatingMap.put(offer.getId(), offer); + + offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + workerSlotsMap = new HashMap<>(); + workerSlots = new ArrayList<>(); + workerSlots.add(new WorkerSlot("h1", 3100)); + workerSlotsMap.put("t1", workerSlots); + + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + + assertTrue((tasksToLaunch.size() == 1) && (tasksToLaunch.get("h1").size() == 1)); + assertTrue(hasResources(FRAMEWORK_ROLE, tasksToLaunch.get("h1").get(0), 0.75 - MesosCommon.DEFAULT_EXECUTOR_CPU, 850 - MesosCommon.DEFAULT_EXECUTOR_MEM_MB)); + assertTrue(hasResources("*", tasksToLaunch.get("h1").get(0), 0.35, 650.0)); + assertTrue(hasCorrectExecutorResources(tasksToLaunch.get("h1"))); + assertEquals(0.4f, TestUtils.calculateAllAvailableScalarResources(offerResourcesPerNode.get("h1"), ResourceType.CPU), 0.01f); + assertEquals(100f, TestUtils.calculateAllAvailableScalarResources(offerResourcesPerNode.get("h1"), ResourceType.MEM), 0.01f); + + // One offer with only reserved resources + offer = TestUtils.buildOfferWithReservationAndPorts("O-1", "h1", 0, 0, 1.5, 1600, 3100, 3101); + rotatingMap.put(offer.getId(), offer); + + offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + workerSlotsMap = new HashMap<>(); + workerSlots = new ArrayList<>(); + workerSlots.add(new WorkerSlot("h1", 3100)); + workerSlotsMap.put("t1", workerSlots); + + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + + assertTrue((tasksToLaunch.size() == 1) && (tasksToLaunch.get("h1").size() == 1)); + assertTrue(hasResources(FRAMEWORK_ROLE, tasksToLaunch.get("h1").get(0), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB)); + assertTrue(hasCorrectExecutorResources(tasksToLaunch.get("h1"))); + assertEquals(TestUtils.calculateAllAvailableScalarResources(offerResourcesPerNode.get("h1"), ResourceType.CPU), 0.4f, 0.01f); + assertEquals(TestUtils.calculateAllAvailableScalarResources(offerResourcesPerNode.get("h1"), ResourceType.MEM), 100f, 0.01f); + + // Offer with Insufficient cpu + offer = TestUtils.buildOfferWithPorts("O-1", "h1", 0, 40000, 3100, 3200); + rotatingMap.put(offer.getId(), offer); + + offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + assertTrue(tasksToLaunch.isEmpty()); + + // Offer with Insufficient Mem for both executor and worker combined + offer = TestUtils.buildOfferWithPorts("O-1", "h1", 24, 900, 3100, 3200); + rotatingMap.put(offer.getId(), offer); + + offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + assertTrue(tasksToLaunch.isEmpty()); + + // Offer with Insufficient Mem for executor + offer = TestUtils.buildOfferWithPorts("O-1", "h1", 24, 1400, 3100, 3200); + rotatingMap.put(offer.getId(), offer); + + offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + assertTrue(tasksToLaunch.isEmpty()); + + // One offer with Insufficient ports + offer = TestUtils.buildOffer("O-1", "h1", 24, 4000); + rotatingMap.put(offer.getId(), offer); + + offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + assertTrue(tasksToLaunch.isEmpty()); + } + + @Test + public void testGetTasksToLaunchForOneTopologyWithMultipleOffersOnSameHost() { + TopologyDetails t1 = TestUtils.constructTopologyDetails("t1", + MesosNimbusTest.DEFAULT_WORKER_COUNT, + MesosCommon.DEFAULT_WORKER_CPU, + MesosCommon.DEFAULT_WORKER_MEM_MB); + Map topologyDetailsMap = new HashMap<>(); + topologyDetailsMap.put("t1", t1); + + Topologies topologies = new Topologies(topologyDetailsMap); + + Offer offer = TestUtils.buildOffer("O-1", "h1", 0, 40000); + rotatingMap.put(offer.getId(), offer); + offer = TestUtils.buildOffer("O-2", "h1", 24, 0); + rotatingMap.put(offer.getId(), offer); + offer = TestUtils.buildOfferWithPorts("O-3", "h1", 0, 0, 3100, 3200); + rotatingMap.put(offer.getId(), offer); + + Map offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + Map> workerSlotsMap = new HashMap<>(); + Collection workerSlots = new ArrayList<>(); + workerSlots.add(new WorkerSlot("h1", 3100)); + workerSlotsMap.put("t1", workerSlots); + Map> tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + assertTrue((tasksToLaunch.size() == 1) && (tasksToLaunch.get("h1").size() == 1)); + List taskInfoList = tasksToLaunch.get("h1"); + assertTrue(hasResources("*", taskInfoList.get(0), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3100l)); + assertTrue(hasCorrectExecutorResources(taskInfoList)); + + offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + workerSlots.add(new WorkerSlot("h1", 3101)); + workerSlots.add(new WorkerSlot("h1", 3102)); + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + taskInfoList = tasksToLaunch.get("h1"); + assertTrue(taskInfoList.size() == 3); + assertTrue(hasResources("*", taskInfoList.get(0), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3100l)); + assertTrue(hasResources("*", taskInfoList.get(1), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3101l)); + assertTrue(hasResources("*", taskInfoList.get(2), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3102l)); + assertTrue(hasCorrectExecutorResources(taskInfoList)); + + TopologyDetails t2 = TestUtils.constructTopologyDetails("t2", + MesosNimbusTest.DEFAULT_WORKER_COUNT, + MesosCommon.DEFAULT_WORKER_CPU, + MesosCommon.DEFAULT_WORKER_MEM_MB); + workerSlots = new ArrayList<>(); + workerSlots.add(new WorkerSlot("h1", 3103)); + workerSlots.add(new WorkerSlot("h1", 3104)); + workerSlotsMap.put("t2", workerSlots); + + topologyDetailsMap = new HashMap<>(); + topologyDetailsMap.put("t1", t1); + topologyDetailsMap.put("t2", t2); + topologies = new Topologies(topologyDetailsMap); + + offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + Map> topologyIDtoTaskInfoMap = getTopologyIDtoTaskInfoMap(tasksToLaunch.get("h1")); + taskInfoList = topologyIDtoTaskInfoMap.get("t1"); + assertTrue(hasResources("*", taskInfoList.get(0), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3100l)); + assertTrue(hasResources("*", taskInfoList.get(1), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3101l)); + assertTrue(hasResources("*", taskInfoList.get(2), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3102l)); + assertTrue(hasCorrectExecutorResources(taskInfoList)); + + taskInfoList = topologyIDtoTaskInfoMap.get("t2"); + assertTrue(hasResources("*", taskInfoList.get(0), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3103l)); + assertTrue(hasResources("*", taskInfoList.get(1), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3104l)); + assertTrue(hasCorrectExecutorResources(taskInfoList)); + } + + @Test + public void testGetTasksToLaunchForOneTopologyWithMultipleOffersAcrossMultipleHosts() { + TopologyDetails t1 = TestUtils.constructTopologyDetails("t1", + MesosNimbusTest.DEFAULT_WORKER_COUNT, + MesosCommon.DEFAULT_WORKER_CPU, + MesosCommon.DEFAULT_WORKER_MEM_MB); + TopologyDetails t2 = TestUtils.constructTopologyDetails("t2", + MesosNimbusTest.DEFAULT_WORKER_COUNT, + MesosCommon.DEFAULT_WORKER_CPU, + MesosCommon.DEFAULT_WORKER_MEM_MB); + + Map topologyDetailsMap = new HashMap<>(); + topologyDetailsMap.put("t1", t1); + topologyDetailsMap.put("t2", t2); + + Topologies topologies = new Topologies(topologyDetailsMap); + + Offer offer = TestUtils.buildOffer("O-H1-1", "h1", 0, 4000); + rotatingMap.put(offer.getId(), offer); + offer = TestUtils.buildOffer("O-H1-2", "h1", 3.21, 0); + rotatingMap.put(offer.getId(), offer); + offer = TestUtils.buildOfferWithPorts("O-H1-3", "h1", 0, 0, 3100, 3102); + rotatingMap.put(offer.getId(), offer); + + offer = TestUtils.buildOffer("O-H2-1", "h2", 0, 4000); + rotatingMap.put(offer.getId(), offer); + offer = TestUtils.buildOffer("O-H2-2", "h2", 3.21, 0); + rotatingMap.put(offer.getId(), offer); + offer = TestUtils.buildOfferWithPorts("O-H2-3", "h2", 0, 0, 3100, 3102); + rotatingMap.put(offer.getId(), offer); + + Map offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + Map> workerSlotsMap = new HashMap<>(); + Map> tasksToLaunch = new HashMap<>(); + + List workerSlots = new ArrayList<>(); + workerSlots.add(new WorkerSlot("h1", 3100)); + workerSlots.add(new WorkerSlot("h2", 3101)); + workerSlots.add(new WorkerSlot("h1", 3102)); + workerSlotsMap.put("t1", workerSlots); + + workerSlots = new ArrayList<>(); + workerSlots.add(new WorkerSlot("h2", 3100)); + workerSlots.add(new WorkerSlot("h1", 3101)); + workerSlots.add(new WorkerSlot("h2", 3102)); + workerSlotsMap.put("t2", workerSlots); + + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + + List taskInfoList = new ArrayList<>(); + for(List til : tasksToLaunch.values()) { + taskInfoList.addAll(til); + } + + Map> topologyIDtoTaskInfoMap = getTopologyIDtoTaskInfoMap(taskInfoList); + taskInfoList = topologyIDtoTaskInfoMap.get("t1"); + assertTrue(hasResources(taskInfoList.get(0), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3100l)); + assertTrue(hasResources(taskInfoList.get(1), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3102l)); + assertTrue(hasResources(taskInfoList.get(2), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3101l)); + assertTrue(hasCorrectExecutorResources(taskInfoList)); + taskInfoList = topologyIDtoTaskInfoMap.get("t2"); + assertTrue(hasResources(taskInfoList.get(0), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3101l)); + assertTrue(hasResources(taskInfoList.get(1), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3100l)); + assertTrue(hasResources(taskInfoList.get(2), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3102l)); + assertTrue(hasCorrectExecutorResources(taskInfoList)); } } diff --git a/storm/src/test/storm/mesos/OfferRoleComparatorTest.java b/storm/src/test/storm/mesos/OfferRoleComparatorTest.java index dcb7980a9..27cf29920 100644 --- a/storm/src/test/storm/mesos/OfferRoleComparatorTest.java +++ b/storm/src/test/storm/mesos/OfferRoleComparatorTest.java @@ -17,12 +17,8 @@ */ package storm.mesos; -import org.apache.mesos.Protos.FrameworkID; import org.apache.mesos.Protos.Offer; -import org.apache.mesos.Protos.OfferID; import org.apache.mesos.Protos.Resource; -import org.apache.mesos.Protos.SlaveID; -import org.apache.mesos.Protos.Value; import org.junit.Test; import java.util.ArrayList; diff --git a/storm/src/test/storm/mesos/TestUtils.java b/storm/src/test/storm/mesos/TestUtils.java index d14b41347..c745ca83c 100644 --- a/storm/src/test/storm/mesos/TestUtils.java +++ b/storm/src/test/storm/mesos/TestUtils.java @@ -20,6 +20,11 @@ import backtype.storm.generated.StormTopology; import backtype.storm.scheduler.TopologyDetails; import org.apache.mesos.Protos; +import storm.mesos.resources.OfferResources; +import storm.mesos.resources.ReservationType; +import storm.mesos.resources.ResourceEntries; +import storm.mesos.resources.ResourceEntry; +import storm.mesos.resources.ResourceType; import storm.mesos.util.MesosCommon; import java.util.ArrayList; @@ -31,6 +36,15 @@ public class TestUtils { + public static TopologyDetails constructTopologyDetails(String topologyName, int numWorkers) { + Map topologyConf1 = new HashMap<>(); + + StormTopology stormTopology = new StormTopology(); + TopologyDetails topologyDetails= new TopologyDetails(topologyName, topologyConf1, stormTopology, numWorkers); + + return topologyDetails; + } + public static TopologyDetails constructTopologyDetails(String topologyName, int numWorkers, double numCpus, double memSize) { Map topologyConf = new HashMap<>(); @@ -51,7 +65,8 @@ public static Protos.Offer buildOffer(String offerId, String hostName, double cp .addAllResources( Arrays.asList( buildScalarResource("cpus", cpus), - buildScalarResourceWithReservation("cpus", 1.0, "dynamicallyReserved"), + buildScalarResourceWithDynamicReservation("cpus", 1.0, "dynamicallyReserved"), + buildScalarResourceWithDynamicReservation("mem", 1.0, "dynamicallyReserved"), buildScalarResource("mem", mem) ) ) @@ -67,7 +82,7 @@ public static Protos.Offer buildOfferWithPorts(String offerId, String hostName, .addAllResources( Arrays.asList( buildScalarResource("cpus", cpus), - buildScalarResourceWithReservation("cpus", 1.0, "dynamicallyReserved"), + buildScalarResourceWithDynamicReservation("cpus", 1.0, "dynamicallyReserved"), buildScalarResource("mem", mem), buildRangeResource("ports", portBegin, portEnd) ) @@ -84,7 +99,7 @@ public static Protos.Offer buildOfferWithReservation(String offerId, String host .addAllResources( Arrays.asList( buildScalarResource("cpus", cpus), - buildScalarResourceWithReservation("cpus", 1.0, "dynamicallyReserved"), + buildScalarResourceWithDynamicReservation("cpus", 1.0, "dynamicallyReserved"), buildScalarResource("mem", mem), buildScalarResourceWithRole("cpus", reservedCpu, "reserved"), buildScalarResourceWithRole("mem", reservedMem, "reserved") @@ -93,6 +108,27 @@ public static Protos.Offer buildOfferWithReservation(String offerId, String host .build(); } + public static Protos.Offer buildOfferWithReservationAndPorts(String offerId, String hostName, double cpus, + double mem, double reservedCpu, double reservedMem, + int portBegin, int portEnd) { + return Protos.Offer.newBuilder() + .setId(Protos.OfferID.newBuilder().setValue(offerId).build()) + .setFrameworkId(Protos.FrameworkID.newBuilder().setValue("derp").build()) + .setSlaveId(Protos.SlaveID.newBuilder().setValue("derp").build()) + .setHostname(hostName) + .addAllResources( + Arrays.asList( + buildScalarResource("cpus", cpus), + buildScalarResourceWithDynamicReservation("cpus", 1.0, "dynamicallyReserved"), + buildScalarResource("mem", mem), + buildScalarResourceWithRole("cpus", reservedCpu, "reserved"), + buildScalarResourceWithRole("mem", reservedMem, "reserved"), + buildRangeResource("ports", portBegin, portEnd) + ) + ) + .build(); + } + public static Protos.Resource buildScalarResource(String name, double value) { return Protos.Resource.newBuilder() .setType(Protos.Value.Type.SCALAR) @@ -114,7 +150,7 @@ public static Protos.Resource buildScalarResourceWithRole(String name, double va .build(); } - public static Protos.Resource buildScalarResourceWithReservation(String name, double value, String role) { + public static Protos.Resource buildScalarResourceWithDynamicReservation(String name, double value, String role) { return Protos.Resource.newBuilder() .setType(Protos.Value.Type.SCALAR) .setScalar(Protos.Value.Scalar.newBuilder() @@ -253,4 +289,41 @@ public static Protos.Offer buildOffer() { .addAllResources(resourceList) .build(); } + + private static double calculateAllAvailableScalarResources(List resourceEntries) { + Double retVal = 0.0; + for (ResourceEntry resourceEntry : resourceEntries) { + retVal += ((ResourceEntries.ScalarResourceEntry) resourceEntry).getValue(); + } + return retVal; + } + + public static List calculateAllAvailableRangeResources(List resourceEntries) { + List retVal = new ArrayList<>(); + for (ResourceEntry resourceEntry : resourceEntries) { + Long begin = ((ResourceEntries.RangeResourceEntry) resourceEntry).getBegin(); + Long end = ((ResourceEntries.RangeResourceEntry) resourceEntry).getEnd(); + for (long i = begin; i <= end; i++) { + retVal.add(i); + } + } + return retVal; + } + + public static double calculateAllAvailableScalarResources(OfferResources offerResources, ResourceType resourceType) { + return calculateAllAvailableScalarResources(offerResources.getAllAvailableResources(resourceType)); + } + + public static List calculateAllAvailableRangeResources(OfferResources offerResources, ResourceType resourceType) { + return calculateAllAvailableRangeResources(offerResources.getAllAvailableResources(resourceType)); + } + + public static double calculateAllAvailableScalarResources(OfferResources offerResources, ResourceType resourceType, ReservationType reservationType) { + return calculateAllAvailableScalarResources(offerResources.getAllAvailableResources(resourceType, reservationType)); + } + + public static List calculateAllAvailableRangeResources(OfferResources offerResources, ResourceType resourceType, ReservationType reservationType) { + return calculateAllAvailableRangeResources(offerResources.getAllAvailableResources(resourceType, reservationType)); + } + } diff --git a/storm/src/test/storm/mesos/resources/OfferResourcesTest.java b/storm/src/test/storm/mesos/resources/OfferResourcesTest.java new file mode 100644 index 000000000..c45d3c630 --- /dev/null +++ b/storm/src/test/storm/mesos/resources/OfferResourcesTest.java @@ -0,0 +1,65 @@ +/** + * 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 storm.mesos.resources; + +import org.apache.mesos.Protos.Offer; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.runners.MockitoJUnitRunner; +import storm.mesos.TestUtils; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.assertTrue; +import storm.mesos.TestUtils; + +@RunWith(MockitoJUnitRunner.class) +public class OfferResourcesTest { + + @Test + public void testToIgnoreDynamicResources() { + ScalarResource scalarResource = new ScalarResource(ResourceType.CPU); + scalarResource.add(new ResourceEntries.ScalarResourceEntry(100.0), ReservationType.STATICALLY_RESERVED); + scalarResource.toString(); + + // Note that buidOffer adds + Offer offer = TestUtils.buildOffer("0-1", "h1", 0, 0); + OfferResources offerResources = new OfferResources(offer); + + assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU) == 0); + assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM) == 0); + + assertTrue(offerResources.getHostName().equals(offer.getHostname())); + assertTrue(offerResources.getSlaveID().equals(offer.getSlaveId())); + + offer = TestUtils.buildOfferWithReservation("offer1", "h1", 2, 1000, 6, 1000); + offerResources = new OfferResources(offer); + assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU) == 8); + assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM) == 2000); + assertTrue(offerResources.getHostName().equals(offer.getHostname())); + assertTrue(offerResources.getSlaveID().equals(offer.getSlaveId())); + + offer = TestUtils.buildOfferWithPorts("offer1", "h1", 2.0, 2000, 3000, 3100); + offerResources = new OfferResources(offer); + assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU) == 2.0); + assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM) == 2000); + List rangeResources = TestUtils.calculateAllAvailableRangeResources(offerResources, ResourceType.PORTS); + assertTrue(rangeResources.size() == 101); + } +} diff --git a/storm/src/test/storm/mesos/schedulers/DefaultSchedulerTest.java b/storm/src/test/storm/mesos/schedulers/DefaultSchedulerTest.java index 2091d0d3b..5ab5efc69 100644 --- a/storm/src/test/storm/mesos/schedulers/DefaultSchedulerTest.java +++ b/storm/src/test/storm/mesos/schedulers/DefaultSchedulerTest.java @@ -39,15 +39,14 @@ import storm.mesos.util.RotatingMap; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; @@ -71,18 +70,6 @@ public class DefaultSchedulerTest { private final String sampleHost = "host1.east"; private final int samplePort = 3100; - - - private TopologyDetails constructTopologyDetails(String topologyName, int numWorkers) { - Map topologyConf1 = new HashMap<>(); - - StormTopology stormTopology = new StormTopology(); - TopologyDetails topologyDetails= new TopologyDetails(topologyName, topologyConf1, stormTopology, numWorkers); - - return topologyDetails; - } - - private Cluster getSpyCluster() { Map supervisors = new HashMap<>(); Map assignmentMap = new HashMap<>(); @@ -232,7 +219,7 @@ public void testAllSlotsAvailableForSchedulingWithOneOffer() { offer = buildOfferWithPorts("offer1", "host-without-supervisor.east", MesosCommon.DEFAULT_WORKER_CPU + MesosCommon.DEFAULT_EXECUTOR_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB + MesosCommon.DEFAULT_EXECUTOR_MEM_MB, samplePort, samplePort + 1); rotatingMap.put(offer.getId(), offer); - TopologyDetails topologyDetails = constructTopologyDetails(sampleTopologyId, 1); + TopologyDetails topologyDetails = TestUtils.constructTopologyDetails(sampleTopologyId, 1); topologyMap.put(sampleTopologyId, topologyDetails); defaultScheduler.prepare(topologyDetails.getConf()); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, new Topologies(topologyMap), @@ -244,7 +231,7 @@ public void testAllSlotsAvailableForSchedulingWithOneOffer() { offer = buildOfferWithPorts("offer1", "host-without-supervisor.east", 10 * MesosCommon.DEFAULT_WORKER_CPU + MesosCommon.DEFAULT_EXECUTOR_CPU, 10 * MesosCommon.DEFAULT_WORKER_MEM_MB + MesosCommon.DEFAULT_EXECUTOR_MEM_MB, samplePort, samplePort + 1); rotatingMap.put(offer.getId(), offer); - topologyDetails = constructTopologyDetails(sampleTopologyId, 10); + topologyDetails = TestUtils.constructTopologyDetails(sampleTopologyId, 10); topologyMap.put(sampleTopologyId, topologyDetails); defaultScheduler.prepare(topologyDetails.getConf()); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, new Topologies(topologyMap), @@ -255,7 +242,7 @@ public void testAllSlotsAvailableForSchedulingWithOneOffer() { offer = buildOfferWithPorts("offer1", "host-without-supervisor.east", 10 * MesosCommon.DEFAULT_WORKER_CPU + MesosCommon.DEFAULT_EXECUTOR_CPU, 2 * MesosCommon.DEFAULT_WORKER_MEM_MB + MesosCommon.DEFAULT_EXECUTOR_MEM_MB, samplePort, samplePort + 1); rotatingMap.put(offer.getId(), offer); - topologyDetails = constructTopologyDetails(sampleTopologyId, 10); + topologyDetails = TestUtils.constructTopologyDetails(sampleTopologyId, 10); topologyMap.put(sampleTopologyId, topologyDetails); defaultScheduler.prepare(topologyDetails.getConf()); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, new Topologies(topologyMap), @@ -266,7 +253,7 @@ public void testAllSlotsAvailableForSchedulingWithOneOffer() { offer = buildOfferWithPorts("offer1", "host-without-supervisor.east", 2 * MesosCommon.DEFAULT_WORKER_CPU + MesosCommon.DEFAULT_EXECUTOR_CPU, 10 * MesosCommon.DEFAULT_WORKER_MEM_MB + MesosCommon.DEFAULT_EXECUTOR_MEM_MB, samplePort, samplePort + 100); rotatingMap.put(offer.getId(), offer); - topologyDetails = constructTopologyDetails(sampleTopologyId, 10); + topologyDetails = TestUtils.constructTopologyDetails(sampleTopologyId, 10); topologyMap.put(sampleTopologyId, topologyDetails); defaultScheduler.prepare(topologyDetails.getConf()); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, new Topologies(topologyMap), @@ -277,7 +264,7 @@ public void testAllSlotsAvailableForSchedulingWithOneOffer() { offer = buildOfferWithPorts("offer1", "host-without-supervisor.east", 20 * MesosCommon.DEFAULT_WORKER_CPU + MesosCommon.DEFAULT_EXECUTOR_CPU, 20 * MesosCommon.DEFAULT_WORKER_MEM_MB + MesosCommon.DEFAULT_EXECUTOR_MEM_MB, samplePort, samplePort + 100); rotatingMap.put(offer.getId(), offer); - topologyDetails = constructTopologyDetails(sampleTopologyId, 10); + topologyDetails = TestUtils.constructTopologyDetails(sampleTopologyId, 10); topologyMap.put(sampleTopologyId, topologyDetails); defaultScheduler.prepare(topologyDetails.getConf()); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, new Topologies(topologyMap), @@ -285,90 +272,139 @@ public void testAllSlotsAvailableForSchedulingWithOneOffer() { assertEquals(workerSlotsAvailableForScheduling.size(), 10); } + private void addToRotatingMap(List offers) { + for (Offer offer:offers) { + rotatingMap.put(offer.getId(), offer); + } + } @Test public void testAllSlotsAvailableForSchedulingWithMultipleOffers() { List workerSlotsAvailableForScheduling; Offer offer; TopologyDetails topologyDetails; + final double DEFAULT_WORKER_CPU = MesosCommon.DEFAULT_WORKER_CPU; + final double DEFAULT_EXECUTOR_CPU = MesosCommon.DEFAULT_EXECUTOR_CPU; + final double DEFAULT_WORKER_MEM = MesosCommon.DEFAULT_WORKER_MEM_MB; + final double DEFAULT_EXECUTOR_MEM = MesosCommon.DEFAULT_EXECUTOR_MEM_MB; + final String sampleHost2 = "host1.west"; /* 10 worker slots are available but offers are fragmented on one host */ - offer = buildOffer("offer1", sampleHost, 0, 1000); - rotatingMap.put(offer.getId(), offer); - offer = buildOffer("offer2", sampleHost, 10, 0); - rotatingMap.put(offer.getId(), offer); - String sampleHost2 = "host1.west"; - offer = buildOffer("offer3", sampleHost2, 0.01, 1000); - rotatingMap.put(offer.getId(), offer); - offer = buildOffer("offer4", sampleHost2, 0.1, 9000); - rotatingMap.put(offer.getId(), offer); - offer = buildOffer("offer5", sampleHost2, 0.91, 9000); - rotatingMap.put(offer.getId(), offer); - offer = buildOfferWithPorts("offer6", sampleHost2, 5 * MesosCommon.DEFAULT_WORKER_CPU + MesosCommon.DEFAULT_EXECUTOR_CPU, - 5 * MesosCommon.DEFAULT_WORKER_MEM_MB + MesosCommon.DEFAULT_EXECUTOR_MEM_MB, samplePort, samplePort + 5); - rotatingMap.put(offer.getId(), offer); - + List offers = new ArrayList<>(); + offers.add(buildOffer("offer1", sampleHost, 0, DEFAULT_EXECUTOR_MEM)); + offers.add(buildOffer("offer2", sampleHost, DEFAULT_EXECUTOR_CPU, 0)); + offers.add(buildOfferWithPorts("offer6", sampleHost, 0, 0, samplePort, samplePort + 5)); + offers.add(buildOffer("offer7", sampleHost, 4 * DEFAULT_WORKER_CPU, 0)); + offers.add(buildOffer("offer8", sampleHost, 0, 4 * DEFAULT_WORKER_MEM)); + + offers.add(buildOffer("offer3", sampleHost2, DEFAULT_EXECUTOR_CPU + DEFAULT_WORKER_CPU, 0)); + offers.add(buildOffer("offer4", sampleHost2, 0, DEFAULT_EXECUTOR_MEM + DEFAULT_WORKER_MEM)); + offers.add(buildOffer("offer5", sampleHost2, 0.01, 10)); + offers.add(buildOfferWithPorts("offer9", sampleHost2, 0, 0, samplePort, samplePort)); + + addToRotatingMap(offers); + + // sampleHost2 - We have enough resources for 1 worker + // sampleHost - We have enough resources for 4 workers topologyMap.clear(); - topologyDetails = constructTopologyDetails(sampleTopologyId, 10); - topologyMap.put(sampleTopologyId, topologyDetails); - defaultScheduler.prepare(topologyDetails.getConf()); - - // Increase available cpu by a tiny fraction in order - offer = buildOfferWithPorts("offer6", sampleHost, 5 * MesosCommon.DEFAULT_WORKER_CPU + 1.1 * MesosCommon.DEFAULT_EXECUTOR_CPU, - 5 * MesosCommon.DEFAULT_WORKER_MEM_MB + MesosCommon.DEFAULT_EXECUTOR_MEM_MB, samplePort, samplePort + 5); - rotatingMap.put(offer.getId(), offer); - - topologyMap.clear(); - topologyDetails = constructTopologyDetails(sampleTopologyId, 10); + topologyDetails = TestUtils.constructTopologyDetails(sampleTopologyId, 10); topologyMap.put(sampleTopologyId, topologyDetails); defaultScheduler.prepare(topologyDetails.getConf()); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, new Topologies(topologyMap), topologiesMissingAssignments); - assertEquals(workerSlotsAvailableForScheduling.size(), 5); // Note that by increasing the executor cpu by a fraction, we are able to get 5 worker slots as we expect + assertEquals(workerSlotsAvailableForScheduling.size(), 5); + // Scenario : Cpu & Mem are available for 5 workers but ports are available only for 3 workers. + // Reduce the number of ports on sampleHost to 2 + offer = buildOfferWithPorts("offer6", sampleHost, 0, 0, samplePort, samplePort + 1); + rotatingMap.put(offer.getId(), offer); + // We now only have resources for 3 workers topologyMap.clear(); - topologyDetails = constructTopologyDetails(sampleTopologyId, 10); + topologyDetails = TestUtils.constructTopologyDetails(sampleTopologyId, 10); topologyMap.put(sampleTopologyId, topologyDetails); defaultScheduler.prepare(topologyDetails.getConf()); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, new Topologies(topologyMap), topologiesMissingAssignments); - assertEquals(workerSlotsAvailableForScheduling.size(), 5); + assertEquals(workerSlotsAvailableForScheduling.size(), 3); - offer = buildOfferWithPorts("offer7", "host2.east", 3 * MesosCommon.DEFAULT_WORKER_CPU + MesosCommon.DEFAULT_EXECUTOR_CPU, - 3 * MesosCommon.DEFAULT_WORKER_MEM_MB + MesosCommon.DEFAULT_EXECUTOR_MEM_MB, samplePort, samplePort + 5); + // Scenario: Mem & Ports are available for 5 workers but cpu is available only for 3 workers. + offer = buildOfferWithPorts("offer6", sampleHost, 0, 0, samplePort, samplePort + 5); + rotatingMap.put(offer.getId(), offer); + offer = buildOffer("offer7", sampleHost, 3 * DEFAULT_WORKER_CPU, 0); rotatingMap.put(offer.getId(), offer); - offer = buildOfferWithPorts("offer8", "host3.east", 100 * MesosCommon.DEFAULT_WORKER_CPU + MesosCommon.DEFAULT_EXECUTOR_CPU, - 100 * MesosCommon.DEFAULT_WORKER_MEM_MB + MesosCommon.DEFAULT_EXECUTOR_MEM_MB, samplePort, samplePort + 10); + workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, + new Topologies(topologyMap), topologiesMissingAssignments); + assertEquals(workerSlotsAvailableForScheduling.size(), 4); + + // Scenario: Cpu & Ports are available for 5 workers but Mem is available only for 3 workers. + offer = buildOfferWithPorts("offer6", sampleHost, 0, 0, samplePort, samplePort + 5); + rotatingMap.put(offer.getId(), offer); + offer = buildOffer("offer7", sampleHost, 4 * DEFAULT_WORKER_CPU, 0); + rotatingMap.put(offer.getId(), offer); + offer = buildOffer("offer8", sampleHost, 0, 2 * DEFAULT_WORKER_MEM); rotatingMap.put(offer.getId(), offer); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, new Topologies(topologyMap), topologiesMissingAssignments); + assertEquals(workerSlotsAvailableForScheduling.size(), 3); + // Scenario: Mem, Cpu & Ports are available for 20 workers. + offers.clear(); + offers.add(buildOfferWithPorts("offer6", sampleHost, 0, 0, samplePort, samplePort + 10)); + offers.add(buildOffer("offer7", sampleHost, 10 * DEFAULT_WORKER_CPU, 0)); + offers.add(buildOffer("offer8", sampleHost, 0, 10 * DEFAULT_WORKER_MEM)); + + offers.add(buildOffer("offer3", sampleHost2, 10 * DEFAULT_WORKER_CPU + DEFAULT_EXECUTOR_CPU, 0)); + offers.add(buildOffer("offer4", sampleHost2, 0, 10 * DEFAULT_WORKER_MEM + DEFAULT_EXECUTOR_MEM)); + offers.add(buildOfferWithPorts("offer9", sampleHost2, 0, 0, samplePort, samplePort + 10)); + addToRotatingMap(offers); + + workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, + new Topologies(topologyMap), topologiesMissingAssignments); assertEquals(workerSlotsAvailableForScheduling.size(), 10); + } + + @Test + public void testWorkerSpreadAcrossHosts() { + List workerSlotsAvailableForScheduling; + TopologyDetails topologyDetails; + final double DEFAULT_WORKER_CPU = MesosCommon.DEFAULT_WORKER_CPU; + final double DEFAULT_EXECUTOR_CPU = MesosCommon.DEFAULT_EXECUTOR_CPU; + final double DEFAULT_WORKER_MEM = MesosCommon.DEFAULT_WORKER_MEM_MB; + final double DEFAULT_EXECUTOR_MEM = MesosCommon.DEFAULT_EXECUTOR_MEM_MB; + final String hostName = "host"; + + topologyMap.clear(); + topologyDetails = TestUtils.constructTopologyDetails(sampleTopologyId, 10); + topologyMap.put(sampleTopologyId, topologyDetails); + defaultScheduler.prepare(topologyDetails.getConf()); + + /* 10 worker slots are available but offers are fragmented on one host */ + List offers = new ArrayList<>(); + offers.add(buildOfferWithPorts("offer1", "0", 10 * DEFAULT_WORKER_CPU + DEFAULT_WORKER_CPU, 10 * DEFAULT_WORKER_MEM + DEFAULT_EXECUTOR_MEM, samplePort, samplePort + 1000)); + offers.add(buildOfferWithPorts("offer2", "1", 10 * DEFAULT_WORKER_CPU + DEFAULT_WORKER_CPU, 10 * DEFAULT_WORKER_MEM + DEFAULT_EXECUTOR_MEM, samplePort, samplePort + 1000)); + offers.add(buildOfferWithPorts("offer3", "2", 1 * DEFAULT_WORKER_CPU + DEFAULT_WORKER_CPU, 10 * DEFAULT_WORKER_MEM + DEFAULT_EXECUTOR_MEM, samplePort, samplePort + 1000)); + offers.add(buildOfferWithPorts("offer4", "3", 10 * DEFAULT_WORKER_CPU + DEFAULT_WORKER_CPU, 10 * DEFAULT_WORKER_MEM + DEFAULT_EXECUTOR_MEM, samplePort, samplePort + 1000)); + addToRotatingMap(offers); + // Make sure that the obtained worker slots are evenly spread across the available resources - Map workerCountPerHostMap = new HashMap<>(); + workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, new Topologies(topologyMap), + topologiesMissingAssignments); + Integer[] expectedWorkerCountPerHost = {3, 3, 1, 3}; + + Integer[] actualWorkerCountPerHost = {0, 0, 0, 0}; for (WorkerSlot workerSlot : workerSlotsAvailableForScheduling) { - Integer tmp = workerCountPerHostMap.get(workerSlot.getNodeId()); - if (tmp == null) { - workerCountPerHostMap.put(workerSlot.getNodeId(), 1); - continue; - } - workerCountPerHostMap.put(workerSlot.getNodeId(), tmp + 1); + MesosWorkerSlot mesosWorkerSlot = (MesosWorkerSlot) workerSlot; + String host = mesosWorkerSlot.getNodeId().split(":")[0]; + actualWorkerCountPerHost[Integer.parseInt(host)]++; } - List expectedWorkerCountArray = Arrays.asList(3, 3, 4); - List actualWorkerCountArray = Arrays.asList( - workerCountPerHostMap.get("host1.east"), - workerCountPerHostMap.get("host2.east"), - workerCountPerHostMap.get("host3.east")); - - Collections.sort(actualWorkerCountArray); - assertEquals(expectedWorkerCountArray, actualWorkerCountArray); + assertArrayEquals(expectedWorkerCountPerHost, actualWorkerCountPerHost); } @Test diff --git a/storm/src/test/storm/mesos/schedulers/OfferResourcesTest.java b/storm/src/test/storm/mesos/schedulers/OfferResourcesTest.java deleted file mode 100644 index 9140ef695..000000000 --- a/storm/src/test/storm/mesos/schedulers/OfferResourcesTest.java +++ /dev/null @@ -1,62 +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 storm.mesos.schedulers; - -import org.apache.mesos.Protos.Offer; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.runners.MockitoJUnitRunner; - -import static org.junit.Assert.assertEquals; -import static storm.mesos.TestUtils.buildOffer; -import static storm.mesos.TestUtils.buildOfferWithPorts; -import static storm.mesos.TestUtils.buildOfferWithReservation; - -@RunWith(MockitoJUnitRunner.class) -public class OfferResourcesTest { - - private final String sampleHost = "host1.east"; - - @Test - public void testOfferResources() throws Exception { - Offer offer = buildOfferWithReservation("offer1", sampleHost, 2, 1000, 6, 1000); - OfferResources offerResources = new OfferResources(offer); - assertEquals(8, offerResources.getCpu(), 0.0f); - assertEquals(2000, offerResources.getMem(), 0.0f); - assertEquals(sampleHost, offerResources.getHostName()); - - offer = buildOffer("offer1", sampleHost, 2.0, 2.0); - offerResources = new OfferResources(offer); - assertEquals(2, offerResources.getCpu(), 0.0); - assertEquals(2, offerResources.getMem(), 0.0); - assertEquals(sampleHost, offerResources.getHostName()); - - offer = buildOfferWithPorts("offer1", sampleHost, 2.0, 2000, 3000, 3100); - offerResources = new OfferResources(offer); - assertEquals(2, offerResources.getCpu(), 0.0); - assertEquals(2000, offerResources.getMem(), 0.0); - assertEquals(true, offerResources.hasPort()); - - offerResources.decCpu(1); - offerResources.decMem(1000); - assertEquals(1, offerResources.getCpu(), 0.0); - assertEquals(1000, offerResources.getMem(), 0.0); - assertEquals(3000, offerResources.getPort()); - assertEquals(3001, offerResources.getPort()); - } -} diff --git a/storm/src/test/storm/mesos/schedulers/SchedulerUtilsTest.java b/storm/src/test/storm/mesos/schedulers/SchedulerUtilsTest.java index 9b60b2abc..b3de8af35 100644 --- a/storm/src/test/storm/mesos/schedulers/SchedulerUtilsTest.java +++ b/storm/src/test/storm/mesos/schedulers/SchedulerUtilsTest.java @@ -24,6 +24,7 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.runners.MockitoJUnitRunner; +import storm.mesos.resources.OfferResources; import storm.mesos.util.MesosCommon; import storm.mesos.util.RotatingMap; From 1f8331a1eaa774a33d0707b40786da110a21203e Mon Sep 17 00:00:00 2001 From: Karthick Duraisamy Soundararaj Date: Tue, 31 May 2016 19:04:09 -0700 Subject: [PATCH 02/12] Trio coding changes for improving allSlotsAvailableForScheduling --- storm/src/main/storm/mesos/MesosNimbus.java | 6 +- .../storm/mesos/resources/OfferResources.java | 30 ++++++ .../storm/mesos/resources/ScalarResource.java | 2 +- .../mesos/schedulers/DefaultScheduler.java | 100 +++++++++--------- .../mesos/schedulers/MesosWorkerSlot.java | 5 + .../mesos/schedulers/SchedulerUtils.java | 41 +++---- .../schedulers/DefaultSchedulerTest.java | 16 +-- 7 files changed, 111 insertions(+), 89 deletions(-) diff --git a/storm/src/main/storm/mesos/MesosNimbus.java b/storm/src/main/storm/mesos/MesosNimbus.java index 65f819b33..e5143ac3c 100644 --- a/storm/src/main/storm/mesos/MesosNimbus.java +++ b/storm/src/main/storm/mesos/MesosNimbus.java @@ -309,11 +309,11 @@ public void resourceOffers(SchedulerDriver driver, List offers) { for (Protos.Offer offer : offers) { if (isHostAccepted(offer.getHostname())) { - LOG.debug("resourceOffers: Recording offer from host: {}, offerId: {}", + LOG.info("resourceOffers: Recording offer from host: {}, offerId: {}", offer.getHostname(), offer.getId().getValue()); _offers.put(offer.getId(), offer); } else { - LOG.debug("resourceOffers: Declining offer from host: {}, offerId: {}", + LOG.info("resourceOffers: Declining offer from host: {}, offerId: {}", offer.getHostname(), offer.getId().getValue()); driver.declineOffer(offer.getId()); } @@ -574,7 +574,7 @@ public Map> getTasksToLaunch(Topologies topologies, boolean autostartLogviewer = !subtractExecutorResources && MesosCommon.autoStartLogViewer(mesosStormConf); - if (!SchedulerUtils.isFit(mesosStormConf, offerResources, topologyDetails, workerPort, !subtractExecutorResources, autostartLogviewer)) { + if (!offerResources.isFit(mesosStormConf, topologyDetails, workerPort, !subtractExecutorResources)) { LOG.error(String.format("Unable to launch worker %s. Required cpu: %f, Required mem: %f. Available OfferResources : %s", workerHost, requiredCpu, requiredMem, offerResources)); continue; diff --git a/storm/src/main/storm/mesos/resources/OfferResources.java b/storm/src/main/storm/mesos/resources/OfferResources.java index ceed063fc..0acbc2c49 100644 --- a/storm/src/main/storm/mesos/resources/OfferResources.java +++ b/storm/src/main/storm/mesos/resources/OfferResources.java @@ -17,9 +17,12 @@ */ package storm.mesos.resources; +import backtype.storm.scheduler.TopologyDetails; import org.apache.mesos.Protos; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import storm.mesos.schedulers.SchedulerUtils; +import storm.mesos.util.MesosCommon; import storm.mesos.util.PrettyProtobuf; import java.util.ArrayList; @@ -155,5 +158,32 @@ public String toString() { return String.format("cpu : %s, memory: %s, ports : %s", availableResources.get(ResourceType.CPU), availableResources.get(ResourceType.MEM), availableResources.get(ResourceType.PORTS)); } + + + public boolean isFit(Map mesosStormConf, TopologyDetails topologyDetails, boolean supervisorExists) { + + double requestedWorkerCpu = MesosCommon.topologyWorkerCpu(mesosStormConf, topologyDetails); + double requestedWorkerMem = MesosCommon.topologyWorkerMem(mesosStormConf, topologyDetails); + + requestedWorkerCpu += supervisorExists ? 0 : MesosCommon.executorCpu(mesosStormConf); + requestedWorkerMem += supervisorExists ? 0 : MesosCommon.executorMem(mesosStormConf); + + return (isAvaliable(ResourceType.CPU, new ResourceEntries.ScalarResourceEntry(requestedWorkerCpu)) && + isAvaliable(ResourceType.MEM, new ResourceEntries.ScalarResourceEntry(requestedWorkerMem)) && + !this.getAllAvailableResources(ResourceType.PORTS).isEmpty()); + } + + public boolean isFit(Map mesosStormConf, TopologyDetails topologyDetails, Long port, boolean supervisorExists) { + + double requestedWorkerCpu = MesosCommon.topologyWorkerCpu(mesosStormConf, topologyDetails); + double requestedWorkerMem = MesosCommon.topologyWorkerMem(mesosStormConf, topologyDetails); + + requestedWorkerCpu += supervisorExists ? 0 : MesosCommon.executorCpu(mesosStormConf); + requestedWorkerMem += supervisorExists ? 0 : MesosCommon.executorMem(mesosStormConf); + + return (isAvaliable(ResourceType.CPU, new ResourceEntries.ScalarResourceEntry(requestedWorkerCpu)) && + isAvaliable(ResourceType.MEM, new ResourceEntries.ScalarResourceEntry(requestedWorkerMem)) && + isAvaliable(ResourceType.PORTS, new ResourceEntries.RangeResourceEntry(port, port))); + } } diff --git a/storm/src/main/storm/mesos/resources/ScalarResource.java b/storm/src/main/storm/mesos/resources/ScalarResource.java index 514b915b6..584c48663 100644 --- a/storm/src/main/storm/mesos/resources/ScalarResource.java +++ b/storm/src/main/storm/mesos/resources/ScalarResource.java @@ -83,7 +83,7 @@ public List removeAndGet(ScalarResourceEntry scalarResourceEntry) public List reserveScalarResource(ResourceType resourceType, ScalarResourceEntry requiredValue) throws ResourceNotAvailabeException { if (totalAvailableResource <= requiredValue.getValue()) { - throw new ResourceNotAvailabeException(String.format("resouceType: {} is not available. Requested {} Available {}", + throw new ResourceNotAvailabeException(String.format("resourceType: {} is not available. Requested {} Available {}", resourceType, requiredValue, totalAvailableResource)); } return removeAndGet(requiredValue); diff --git a/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java b/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java index a044fc1e8..36056edab 100644 --- a/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java +++ b/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java @@ -58,6 +58,52 @@ public void prepare(Map conf) { mesosStormConf = conf; } + private List getMesosWorkerSlots(Map offerResourcesPerNode, + Collection nodesWithExistingSupervisors, + TopologyDetails topologyDetails) { + + double requestedWorkerCpu = MesosCommon.topologyWorkerCpu(mesosStormConf, topologyDetails); + double requestedWorkerMem = MesosCommon.topologyWorkerMem(mesosStormConf, topologyDetails); + + List mesosWorkerSlots = new ArrayList<>(); + boolean slotFound = true; + int slotsNeeded = topologyDetails.getNumWorkers(); + + while (slotFound && slotsNeeded > 0) { + slotFound = false; + for (String currentNode : offerResourcesPerNode.keySet()) { + OfferResources offerResources = offerResourcesPerNode.get(currentNode); + + boolean supervisorExists = nodesWithExistingSupervisors.contains(currentNode); + + if (!offerResources.isFit(mesosStormConf, topologyDetails, supervisorExists)) { + log.info("{} is not a fit for {} requestedWorkerCpu: {} requestedWorkerMem: {}", + offerResources.toString(), topologyDetails.getId(), requestedWorkerCpu, requestedWorkerMem); + continue; + } + + log.info("{} is a fit for {} requestedWorkerCpu: {} requestedWorkerMem: {}", offerResources.toString(), + topologyDetails.getId(), requestedWorkerCpu, requestedWorkerMem); + nodesWithExistingSupervisors.add(currentNode); + MesosWorkerSlot mesosWorkerSlot; + + try { + mesosWorkerSlot = SchedulerUtils.createMesosWorkerSlot(mesosStormConf, offerResources, topologyDetails, supervisorExists); + } catch (ResourceNotAvailabeException rexp) { + log.warn(rexp.getMessage()); + continue; + } + + mesosWorkerSlots.add(mesosWorkerSlot); + slotFound = true; + if (--slotsNeeded == 0) { + break; + } + } + } + + return mesosWorkerSlots; + } /* * Different topologies have different resource requirements in terms of cpu and memory. So when Mesos asks * this scheduler for a list of available worker slots, we create "MesosWorkerSlot" and store them into mesosWorkerSlotMap. @@ -93,8 +139,6 @@ public List allSlotsAvailableForScheduling(RotatingMap allSlotsAvailableForScheduling(RotatingMap 0 && slotFound) { - slotFound = false; - for (String currentNode : offerResourcesPerNode.keySet()) { - OfferResources offerResources = offerResourcesPerNode.get(currentNode); - - boolean supervisorExists = nodesWithExistingSupervisors.contains(currentNode); - boolean isFit = false; - List availablePorts = SchedulerUtils.getPorts(offerResources, 1); - - if (!availablePorts.isEmpty()) { - isFit = SchedulerUtils.isFit(mesosStormConf, offerResources, topologyDetails, availablePorts.get(0).getBegin(), supervisorExists, MesosCommon.autoStartLogViewer(mesosStormConf)); - } - - if (!isFit) { - log.info("{} is not a fit for {} requestedWorkerCpu: {} requestedWorkerMem: {}", - offerResources.toString(), topologyDetails.getId(), requestedWorkerCpu, requestedWorkerMem); - continue; - } - - log.info("{} is a fit for {} requestedWorkerCpu: {} requestedWorkerMem: {}", offerResources.toString(), - topologyDetails.getId(), requestedWorkerCpu, requestedWorkerMem); - nodesWithExistingSupervisors.add(currentNode); - MesosWorkerSlot mesosWorkerSlot; - - try { - mesosWorkerSlot = SchedulerUtils.createMesosWorkerSlot(mesosStormConf, offerResources, topologyDetails, supervisorExists); - } catch (ResourceNotAvailabeException rexp) { - log.warn(rexp.getMessage()); - continue; - } - - String slotId = String.format("%s:%s", mesosWorkerSlot.getNodeId(), mesosWorkerSlot.getPort()); - mesosWorkerSlotMap.put(slotId, mesosWorkerSlot); - allSlots.add(mesosWorkerSlot); - slotFound = true; - if (--slotsNeeded == 0) { - break; - } - } + List mesosWorkerSlotList = getMesosWorkerSlots(offerResourcesPerNode, nodesWithExistingSupervisors, topologyDetails); + for (MesosWorkerSlot mesosWorkerSlot : mesosWorkerSlotList) { + String slotId = String.format("%s:%s", mesosWorkerSlot.getNodeId(), mesosWorkerSlot.getPort()); + mesosWorkerSlotMap.put(slotId, mesosWorkerSlot); + allSlots.add(mesosWorkerSlot); } + log.info("Number of available slots for {} : {}", topologyDetails.getId(), allSlots.size()); } log.info("Number of available slots: {}", allSlots.size()); - if (log.isDebugEnabled()) { - for (WorkerSlot slot : allSlots) { - log.debug("available slot: {}", slot); - } - } return allSlots; } diff --git a/storm/src/main/storm/mesos/schedulers/MesosWorkerSlot.java b/storm/src/main/storm/mesos/schedulers/MesosWorkerSlot.java index 0ebc6c0d5..99ff34c0c 100644 --- a/storm/src/main/storm/mesos/schedulers/MesosWorkerSlot.java +++ b/storm/src/main/storm/mesos/schedulers/MesosWorkerSlot.java @@ -30,4 +30,9 @@ public MesosWorkerSlot(String nodeId, Number port, String topologyId) { public String getTopologyId() { return this.topologyId; } + + @Override + public String toString() { + return super.getNodeId() + ":" + super.getPort() + " topology: " + topologyId; + } } diff --git a/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java b/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java index a7b55f975..1c6bad7af 100644 --- a/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java +++ b/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java @@ -58,24 +58,6 @@ public static List getPorts(OfferResources offerResources, i return retVal; } - public static boolean isFit(Map mesosStormConf, - OfferResources offerResources, - TopologyDetails topologyDetails, - Long port, - boolean supervisorExists, - boolean autoStartLogviewer) { - - double requestedWorkerCpu = MesosCommon.topologyWorkerCpu(mesosStormConf, topologyDetails); - double requestedWorkerMem = MesosCommon.topologyWorkerMem(mesosStormConf, topologyDetails); - - requestedWorkerCpu += supervisorExists ? 0 : MesosCommon.executorCpu(mesosStormConf); - requestedWorkerMem += supervisorExists ? 0 : MesosCommon.executorMem(mesosStormConf); - - return (offerResources.isAvaliable(ResourceType.CPU, new ScalarResourceEntry(requestedWorkerCpu)) && - offerResources.isAvaliable(ResourceType.MEM, new ScalarResourceEntry(requestedWorkerMem)) && - offerResources.isAvaliable(ResourceType.PORTS, new RangeResourceEntry(port, port))); - } - public static MesosWorkerSlot createMesosWorkerSlot(Map mesosStormConf, @@ -93,32 +75,33 @@ public static MesosWorkerSlot createMesosWorkerSlot(Map mesosStormConf, offerResources.reserve(ResourceType.MEM, new ScalarResourceEntry(requestedWorkerMem)); List ports = getPorts(offerResources, 1); + if (ports.isEmpty()) { + throw new ResourceNotAvailabeException("No ports available to create MesosWorkerSlot."); + } offerResources.reserve(ResourceType.PORTS, ports.get(0)); return new MesosWorkerSlot(offerResources.getHostName(), ports.get(0).getBegin(), topologyDetails.getId()); } /** - * Method checks if all topologies that need assignment already have supervisor running on the node where the Offer - * comes from. Required for more accurate available resource calculation where we can exclude supervisor's demand from - * the Offer. - * Unfortunately because of WorkerSlot type is not topology agnostic, we need to exclude supervisor's resources only - * in case where ALL topologies in 'allSlotsAvailableForScheduling' method satisfy condition of supervisor existence - * @param offerHost hostname corresponding to the offer - * @param existingSupervisors Supervisors which already placed on the node for the Offer - * @param topologyId Topology id for which we are checking if the supervisor exists already + * Check if this topology already has a supervisor running on the node where the Offer + * comes from. Required to account for supervisor/mesos-executor's resource needs. + * Note that there is one-and-only-one supervisor per topology per node. + * + * @param offerHost host that sent this Offer + * @param existingSupervisors List of supervisors which already exist on the Offer's node + * @param topologyId ID of topology requiring assignment * @return boolean value indicating supervisor existence */ public static boolean supervisorExists(String offerHost, Collection existingSupervisors, String topologyId) { - boolean supervisorExists = false; String expectedSupervisorId = MesosCommon.supervisorId(offerHost, topologyId); for (SupervisorDetails supervisorDetail : existingSupervisors) { if (supervisorDetail.getId().equals(expectedSupervisorId)) { - supervisorExists = true; + return true; } } - return supervisorExists; + return false; } public static Map> getOfferResourcesListPerNode(RotatingMap offers) { diff --git a/storm/src/test/storm/mesos/schedulers/DefaultSchedulerTest.java b/storm/src/test/storm/mesos/schedulers/DefaultSchedulerTest.java index 5ab5efc69..846867966 100644 --- a/storm/src/test/storm/mesos/schedulers/DefaultSchedulerTest.java +++ b/storm/src/test/storm/mesos/schedulers/DefaultSchedulerTest.java @@ -198,14 +198,14 @@ public void testAllSlotsAvailableForSchedulingWithOneOffer() { rotatingMap.put(offer.getId(), offer); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, topologies, topologiesMissingAssignments); - assertEquals(workerSlotsAvailableForScheduling.size(), 1); + assertEquals(1, workerSlotsAvailableForScheduling.size()); /* Case 2 - Supervisor does not exists for topology test-topology1-65-1442255385 on the host */ offer = buildOfferWithPorts("offer1", "host-without-supervisor.east", 0.1, 200, samplePort, samplePort + 1); rotatingMap.put(offer.getId(), offer); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, topologies, topologiesMissingAssignments); - assertEquals(workerSlotsAvailableForScheduling.size(), 0); + assertEquals(0, workerSlotsAvailableForScheduling.size()); /* Case 3 - Supervisor exists for topology test-topology1-65-1442255385 on the host & offer has additional resources for supervisor */ offer = buildOfferWithPorts("offer1", "host-without-supervisor.east", 0.1 + MesosCommon.DEFAULT_EXECUTOR_CPU, 200 + MesosCommon.DEFAULT_EXECUTOR_MEM_MB, @@ -213,7 +213,7 @@ public void testAllSlotsAvailableForSchedulingWithOneOffer() { rotatingMap.put(offer.getId(), offer); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, topologies, topologiesMissingAssignments); - assertEquals(workerSlotsAvailableForScheduling.size(), 1); + assertEquals(1, workerSlotsAvailableForScheduling.size()); /* Test default values for worker cpu and memory - This is to make sure that we account for default worker cpu and memory when the user does not pass MesosCommon.DEFAULT_WORKER_CPU && MesosCommon.DEFAULT_WORKER_MEM */ offer = buildOfferWithPorts("offer1", "host-without-supervisor.east", MesosCommon.DEFAULT_WORKER_CPU + MesosCommon.DEFAULT_EXECUTOR_CPU, @@ -224,7 +224,7 @@ public void testAllSlotsAvailableForSchedulingWithOneOffer() { defaultScheduler.prepare(topologyDetails.getConf()); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, new Topologies(topologyMap), topologiesMissingAssignments); - assertEquals(workerSlotsAvailableForScheduling.size(), 1); + assertEquals(1, workerSlotsAvailableForScheduling.size()); /* More than 1 worker slot is required - Plenty of memory & cpu is available, only two ports are available */ @@ -236,7 +236,7 @@ public void testAllSlotsAvailableForSchedulingWithOneOffer() { defaultScheduler.prepare(topologyDetails.getConf()); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, new Topologies(topologyMap), topologiesMissingAssignments); - assertEquals(workerSlotsAvailableForScheduling.size(), 2); + assertEquals(2, workerSlotsAvailableForScheduling.size()); /* More than 1 worker slot is required - Plenty of ports & cpu is available, but memory is available for only two workers */ offer = buildOfferWithPorts("offer1", "host-without-supervisor.east", 10 * MesosCommon.DEFAULT_WORKER_CPU + MesosCommon.DEFAULT_EXECUTOR_CPU, @@ -247,7 +247,7 @@ public void testAllSlotsAvailableForSchedulingWithOneOffer() { defaultScheduler.prepare(topologyDetails.getConf()); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, new Topologies(topologyMap), topologiesMissingAssignments); - assertEquals(workerSlotsAvailableForScheduling.size(), 2); + assertEquals(2, workerSlotsAvailableForScheduling.size()); /* More than 1 worker slot is required - Plenty of ports & memory are available, but cpu is available for only two workers */ offer = buildOfferWithPorts("offer1", "host-without-supervisor.east", 2 * MesosCommon.DEFAULT_WORKER_CPU + MesosCommon.DEFAULT_EXECUTOR_CPU, @@ -258,7 +258,7 @@ public void testAllSlotsAvailableForSchedulingWithOneOffer() { defaultScheduler.prepare(topologyDetails.getConf()); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, new Topologies(topologyMap), topologiesMissingAssignments); - assertEquals(workerSlotsAvailableForScheduling.size(), 2); + assertEquals(2, workerSlotsAvailableForScheduling.size()); /* 10 worker slots are required - Plenty of cpu, memory & ports are available */ offer = buildOfferWithPorts("offer1", "host-without-supervisor.east", 20 * MesosCommon.DEFAULT_WORKER_CPU + MesosCommon.DEFAULT_EXECUTOR_CPU, @@ -269,7 +269,7 @@ public void testAllSlotsAvailableForSchedulingWithOneOffer() { defaultScheduler.prepare(topologyDetails.getConf()); workerSlotsAvailableForScheduling = defaultScheduler.allSlotsAvailableForScheduling(rotatingMap, existingSupervisors, new Topologies(topologyMap), topologiesMissingAssignments); - assertEquals(workerSlotsAvailableForScheduling.size(), 10); + assertEquals(10, workerSlotsAvailableForScheduling.size()); } private void addToRotatingMap(List offers) { From dcb443c1c511b567ada8bfdf22be7ab685313d20 Mon Sep 17 00:00:00 2001 From: Karthick Duraisamy Soundararaj Date: Mon, 6 Jun 2016 11:29:24 -0700 Subject: [PATCH 03/12] Remove additional logs for testing plus and optimize import statements --- storm/src/main/storm/mesos/MesosNimbus.java | 22 +++++++++---------- .../src/main/storm/mesos/MesosSupervisor.java | 4 ---- .../src/main/storm/mesos/TaskAssignments.java | 7 +++--- .../storm/mesos/resources/OfferResources.java | 2 -- .../storm/mesos/resources/ScalarResource.java | 2 +- .../mesos/schedulers/DefaultScheduler.java | 8 +++---- .../mesos/schedulers/SchedulerUtils.java | 5 ++--- 7 files changed, 19 insertions(+), 31 deletions(-) diff --git a/storm/src/main/storm/mesos/MesosNimbus.java b/storm/src/main/storm/mesos/MesosNimbus.java index e5143ac3c..035f88959 100644 --- a/storm/src/main/storm/mesos/MesosNimbus.java +++ b/storm/src/main/storm/mesos/MesosNimbus.java @@ -24,7 +24,6 @@ import backtype.storm.scheduler.Topologies; import backtype.storm.scheduler.TopologyDetails; import backtype.storm.scheduler.WorkerSlot; -import clojure.lang.MapEntry; import com.google.common.base.Optional; import com.google.protobuf.ByteString; import org.apache.commons.lang3.StringUtils; @@ -53,7 +52,6 @@ import org.yaml.snakeyaml.Yaml; import storm.mesos.resources.OfferResources; import storm.mesos.resources.ReservationType; -import storm.mesos.resources.ResourceEntries; import storm.mesos.resources.ResourceEntries.RangeResourceEntry; import storm.mesos.resources.ResourceEntries.ScalarResourceEntry; import storm.mesos.resources.ResourceEntry; @@ -61,7 +59,6 @@ import storm.mesos.resources.ResourceType; import storm.mesos.schedulers.DefaultScheduler; import storm.mesos.schedulers.IMesosStormScheduler; -import storm.mesos.schedulers.SchedulerUtils; import storm.mesos.shims.CommandLineShimFactory; import storm.mesos.shims.ICommandLineShim; import storm.mesos.shims.LocalStateShim; @@ -117,6 +114,7 @@ public class MesosNimbus implements INimbus { public static final String CONF_MESOS_FRAMEWORK_NAME = "mesos.framework.name"; public static final String CONF_MESOS_PREFER_RESERVED_RESOURCES = "mesos.prefer.reserved.resources"; public static final String CONF_MESOS_CONTAINER_DOCKER_IMAGE = "mesos.container.docker.image"; + public static final String CONF_MESOS_SUPERVISOR_STORM_LOCAL_DIR = "mesos.supervisor.storm.local.dir"; public static final String FRAMEWORK_ID = "FRAMEWORK_ID"; private static final Logger LOG = LoggerFactory.getLogger(MesosNimbus.class); private final Object _offersLock = new Object(); @@ -309,11 +307,12 @@ public void resourceOffers(SchedulerDriver driver, List offers) { for (Protos.Offer offer : offers) { if (isHostAccepted(offer.getHostname())) { - LOG.info("resourceOffers: Recording offer from host: {}, offerId: {}", + // TODO(ksoundararaj): Should we record the following as info instead of debug + LOG.debug("resourceOffers: Recording offer from host: {}, offerId: {}", offer.getHostname(), offer.getId().getValue()); _offers.put(offer.getId(), offer); } else { - LOG.info("resourceOffers: Declining offer from host: {}, offerId: {}", + LOG.debug("resourceOffers: Declining offer from host: {}, offerId: {}", offer.getHostname(), offer.getId().getValue()); driver.declineOffer(offer.getId()); } @@ -571,8 +570,7 @@ public Map> getTasksToLaunch(Topologies topologies, String executorName = "storm-supervisor | " + topologyAndNodeId; String taskName = "storm-worker | " + topologyAndNodeId + ":" + slot.getPort(); String executorDataStr = JSONValue.toJSONString(executorData); - - boolean autostartLogviewer = !subtractExecutorResources && MesosCommon.autoStartLogViewer(mesosStormConf); + String extraConfig = ""; if (!offerResources.isFit(mesosStormConf, topologyDetails, workerPort, !subtractExecutorResources)) { LOG.error(String.format("Unable to launch worker %s. Required cpu: %f, Required mem: %f. Available OfferResources : %s", @@ -617,6 +615,11 @@ public Map> getTasksToLaunch(Topologies topologies, executorPortResources.add(createMesosRangeResource(ResourceType.PORTS, logViewerPortList)); } */ + String supervisorStormLocalDir = (String) mesosStormConf.get(CONF_MESOS_SUPERVISOR_STORM_LOCAL_DIR); + if (supervisorStormLocalDir != null) { + extraConfig += String.format(" -c storm.local.dir=%s", supervisorStormLocalDir); + } + scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.CPU, new ScalarResourceEntry(workerCpu)); workerCpuResources.addAll(createMesosScalarResource(ResourceType.CPU, scalarResourceEntryList)); scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.MEM, new ScalarResourceEntry(workerMem)); @@ -632,11 +635,6 @@ public Map> getTasksToLaunch(Topologies topologies, } - String extraConfig = ""; - if (autostartLogviewer) { - extraConfig = getLogViewerConfig(); - } - ExecutorInfo.Builder executorInfoBuilder = getExecutorInfoBuilder(topologyDetails, executorDataStr, executorName, executorCpuResources, executorMemResources, executorPortResources, extraConfig); TaskID taskId = TaskID.newBuilder() diff --git a/storm/src/main/storm/mesos/MesosSupervisor.java b/storm/src/main/storm/mesos/MesosSupervisor.java index ab287b17b..375d3b9cc 100644 --- a/storm/src/main/storm/mesos/MesosSupervisor.java +++ b/storm/src/main/storm/mesos/MesosSupervisor.java @@ -20,7 +20,6 @@ import backtype.storm.scheduler.ISupervisor; import backtype.storm.utils.Utils; import clojure.lang.PersistentVector; - import org.apache.mesos.Executor; import org.apache.mesos.ExecutorDriver; import org.apache.mesos.MesosExecutorDriver; @@ -35,11 +34,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import storm.mesos.logviewer.LogViewerController; -import storm.mesos.shims.ILocalStateShim; -import storm.mesos.shims.LocalStateShim; import storm.mesos.util.MesosCommon; -import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; diff --git a/storm/src/main/storm/mesos/TaskAssignments.java b/storm/src/main/storm/mesos/TaskAssignments.java index a0ee9c00a..0716744b8 100644 --- a/storm/src/main/storm/mesos/TaskAssignments.java +++ b/storm/src/main/storm/mesos/TaskAssignments.java @@ -17,15 +17,14 @@ */ package storm.mesos; +import org.apache.mesos.Protos.TaskID; +import storm.mesos.util.MesosCommon; + import java.io.Serializable; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import org.apache.mesos.Protos.TaskID; - -import storm.mesos.util.MesosCommon; - /** * Tracks the Mesos Tasks / Storm Worker Processes that have been assigned * to this MesosSupervisor instance. diff --git a/storm/src/main/storm/mesos/resources/OfferResources.java b/storm/src/main/storm/mesos/resources/OfferResources.java index 0acbc2c49..76c8faa73 100644 --- a/storm/src/main/storm/mesos/resources/OfferResources.java +++ b/storm/src/main/storm/mesos/resources/OfferResources.java @@ -21,7 +21,6 @@ import org.apache.mesos.Protos; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import storm.mesos.schedulers.SchedulerUtils; import storm.mesos.util.MesosCommon; import storm.mesos.util.PrettyProtobuf; @@ -77,7 +76,6 @@ public void add(Protos.Offer offer) { } ResourceType resourceType = ResourceType.of(r.getName()); - log.info("Processing Resource {}", r.getName()); ReservationType reservationType = (r.getRole().equals("*")) ? ReservationType.UNRESERVED : ReservationType.STATICALLY_RESERVED; diff --git a/storm/src/main/storm/mesos/resources/ScalarResource.java b/storm/src/main/storm/mesos/resources/ScalarResource.java index 584c48663..23382f42c 100644 --- a/storm/src/main/storm/mesos/resources/ScalarResource.java +++ b/storm/src/main/storm/mesos/resources/ScalarResource.java @@ -129,7 +129,7 @@ public String toString() { availableResourcesByResourceTypeList.add(String.format("%s : %s", entry.getKey(), entry.getValue().getValue())); } String tmp = String.join(", ", availableResourcesByResourceTypeList); - return String.format("Resouce %s - Total available : %f Total available by reservation type : [ %s ]", resourceType.toString(), totalAvailableResource, tmp); + return String.format("Resource %s - Total available : %f Total available by reservation type : [ %s ]", resourceType.toString(), totalAvailableResource, tmp); } private List removeAndGet(ScalarResourceEntry scalarResourceEntry, Collection reservationTypesListByPriority) throws ResourceNotAvailabeException { diff --git a/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java b/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java index 36056edab..bfd35a195 100644 --- a/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java +++ b/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java @@ -27,10 +27,7 @@ import org.apache.mesos.Protos; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import storm.mesos.MesosNimbus; import storm.mesos.resources.OfferResources; -import storm.mesos.resources.RangeResource; -import storm.mesos.resources.ResourceEntries; import storm.mesos.resources.ResourceNotAvailabeException; import storm.mesos.util.MesosCommon; import storm.mesos.util.RotatingMap; @@ -125,8 +122,9 @@ public List allSlotsAvailableForScheduling(RotatingMap existingSupervisors, Topologies topologies, Set topologiesMissingAssignments) { if (topologiesMissingAssignments.isEmpty()) { - log.info("Not Declining all offers that are currently buffered because no topologies need assignments"); - //offers.clear(); + log.info("Declining all offers that are currently buffered because no topologies need assignments"); + // TODO(ksoundararaj): Do we need to clear offers not that consolidate resources? + offers.clear(); return new ArrayList<>(); } diff --git a/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java b/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java index 1c6bad7af..db01719ab 100644 --- a/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java +++ b/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java @@ -23,8 +23,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import storm.mesos.resources.OfferResources; -import storm.mesos.resources.RangeResource; -import storm.mesos.resources.ResourceEntry; import storm.mesos.resources.ResourceNotAvailabeException; import storm.mesos.resources.ResourceType; import storm.mesos.util.MesosCommon; @@ -36,7 +34,8 @@ import java.util.List; import java.util.Map; -import static storm.mesos.resources.ResourceEntries.*; +import static storm.mesos.resources.ResourceEntries.RangeResourceEntry; +import static storm.mesos.resources.ResourceEntries.ScalarResourceEntry; public class SchedulerUtils { From fa3bba33aab4ca31c327ad3bca8f58e82b62ad5d Mon Sep 17 00:00:00 2001 From: Karthick Duraisamy Soundararaj Date: Mon, 6 Jun 2016 15:02:53 -0700 Subject: [PATCH 04/12] Using java8 specific function String.join causes travis to fail with the following error while building java7 binary. ``` testGetTasksToLaunchForOneTopologyWithOneOffer(storm.mesos.MesosNimbusTest) Time elapsed: 0.158 sec <<< ERROR! java.lang.NoSuchMethodError: java.lang.String.join(Ljava/lang/CharSequence;Ljava/lang/Iterable;)Ljava/lang/String; ``` Fixing ^^ by using org.apache.commons.lang3.StringUtils.join instead of java.lang.String.join --- storm/src/main/storm/mesos/resources/ScalarResource.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/storm/src/main/storm/mesos/resources/ScalarResource.java b/storm/src/main/storm/mesos/resources/ScalarResource.java index 23382f42c..5773bf0c9 100644 --- a/storm/src/main/storm/mesos/resources/ScalarResource.java +++ b/storm/src/main/storm/mesos/resources/ScalarResource.java @@ -17,6 +17,8 @@ */ package storm.mesos.resources; +import org.apache.commons.lang3.StringUtils; + import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -128,7 +130,7 @@ public String toString() { for (Map.Entry entry: availableResourcesByReservationType.entrySet()) { availableResourcesByResourceTypeList.add(String.format("%s : %s", entry.getKey(), entry.getValue().getValue())); } - String tmp = String.join(", ", availableResourcesByResourceTypeList); + String tmp = StringUtils.join(", ", availableResourcesByResourceTypeList); return String.format("Resource %s - Total available : %f Total available by reservation type : [ %s ]", resourceType.toString(), totalAvailableResource, tmp); } From 6f526af38ea5120dd36c4e3ab74295cc32966ca0 Mon Sep 17 00:00:00 2001 From: Karthick Duraisamy Soundararaj Date: Fri, 10 Jun 2016 13:45:41 -0700 Subject: [PATCH 05/12] Addressing First Round of code review comments from Erik Weathers and Jessica Hartog. Still TODO: 1. Refactor MesosNimbus.getTasksToLaunch as per Erik's comment 2. Refactor SchedulerUtils.getPorts as per Jessica's comment --- storm/src/main/storm/mesos/MesosNimbus.java | 118 +++++++++--------- .../storm/mesos/resources/OfferResources.java | 23 ++-- .../storm/mesos/resources/RangeResource.java | 83 ++++++------ .../main/storm/mesos/resources/Resource.java | 6 +- .../mesos/resources/ResourceEntries.java | 20 ++- ...ava => ResourceNotAvailableException.java} | 4 +- .../storm/mesos/resources/ResourceType.java | 6 +- .../storm/mesos/resources/ScalarResource.java | 26 ++-- .../mesos/schedulers/DefaultScheduler.java | 14 +-- .../mesos/schedulers/MesosWorkerSlot.java | 2 +- .../mesos/schedulers/SchedulerUtils.java | 28 +---- .../main/storm/mesos/util/MesosCommon.java | 18 +-- .../main/storm/mesos/util/PrettyProtobuf.java | 2 +- .../main/storm/mesos/util/RotatingMap.java | 2 +- .../src/test/storm/mesos/MesosCommonTest.java | 50 ++++---- .../src/test/storm/mesos/MesosNimbusTest.java | 27 ++-- .../mesos/resources/OfferResourcesTest.java | 16 +-- .../mesos/schedulers/SchedulerUtilsTest.java | 34 ----- 18 files changed, 221 insertions(+), 258 deletions(-) rename storm/src/main/storm/mesos/resources/{ResourceNotAvailabeException.java => ResourceNotAvailableException.java} (87%) diff --git a/storm/src/main/storm/mesos/MesosNimbus.java b/storm/src/main/storm/mesos/MesosNimbus.java index 035f88959..4601d7317 100644 --- a/storm/src/main/storm/mesos/MesosNimbus.java +++ b/storm/src/main/storm/mesos/MesosNimbus.java @@ -55,7 +55,7 @@ import storm.mesos.resources.ResourceEntries.RangeResourceEntry; import storm.mesos.resources.ResourceEntries.ScalarResourceEntry; import storm.mesos.resources.ResourceEntry; -import storm.mesos.resources.ResourceNotAvailabeException; +import storm.mesos.resources.ResourceNotAvailableException; import storm.mesos.resources.ResourceType; import storm.mesos.schedulers.DefaultScheduler; import storm.mesos.schedulers.IMesosStormScheduler; @@ -178,7 +178,7 @@ public String getHostName(Map map, String nodeId) { public void prepare(Map conf, String localDir) { try { initializeMesosStormConf(conf, localDir); - startLocalServer(); + startLocalHttpServer(); MesosSchedulerDriver driver = createMesosDriver(); @@ -204,15 +204,16 @@ void initializeMesosStormConf(Map conf, String localDir) { try { _state = new LocalStateShim(localDir); } catch (IOException exp) { - // TODO(ksoundararaj) : Should we exit here? - LOG.error("Encounted IOException while setting up LocalState at {} : {}", localDir, exp); + throw new RuntimeException(String.format("Encountered IOException while setting up LocalState at %s : %s", localDir, exp)); } + _allowedHosts = listIntoSet((List) conf.get(CONF_MESOS_ALLOWED_HOSTS)); _disallowedHosts = listIntoSet((List) conf.get(CONF_MESOS_DISALLOWED_HOSTS)); Boolean preferReservedResources = (Boolean) conf.get(CONF_MESOS_PREFER_RESERVED_RESOURCES); if (preferReservedResources != null) { _preferReservedResources = preferReservedResources; } + _container = Optional.fromNullable((String) conf.get(CONF_MESOS_CONTAINER_DOCKER_IMAGE)); _scheduler = new NimbusScheduler(this); @@ -227,8 +228,7 @@ void initializeMesosStormConf(Map conf, String localDir) { try { mesosStormConf.put(Config.NIMBUS_HOST, MesosCommon.getNimbusHost(mesosStormConf)); } catch (UnknownHostException exp) { - LOG.error("Exception while configuring nimbus host: {}", exp); - // TODO(ksoundararaj): Should we exit here? + throw new RuntimeException(String.format("Exception while configuring nimbus host: %s", exp)); } Path pathToDumpConfig = Paths.get(_generatedConfPath.toString(), "storm.yaml"); @@ -243,7 +243,7 @@ void initializeMesosStormConf(Map conf, String localDir) { } @SuppressWarnings("unchecked") - protected void startLocalServer() throws Exception { + protected void startLocalHttpServer() throws Exception { createLocalServerPort(); setupHttpServer(); } @@ -414,6 +414,12 @@ public void assignSlots(Topologies topologies, Map offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(_offers); Map> tasksToLaunchPerNode = getTasksToLaunch(topologies, slotsForTopologiesNeedingAssignments, offerResourcesPerNode); @@ -431,16 +437,37 @@ public void assignSlots(Topologies topologies, Map createMesosScalarResource(ResourceType resourceType, List scalarResourceEntryList) { + private List createMesosScalarResourceList(ResourceType resourceType, List scalarResourceEntryList) { List retVal = new ArrayList<>(); ScalarResourceEntry scalarResourceEntry = null; @@ -486,9 +513,7 @@ String getFullConfigUri() { private ExecutorInfo.Builder getExecutorInfoBuilder(TopologyDetails details, String executorDataStr, String executorName, - List executorCpuResources, - List executorMemResources, - List executorPortsResources, + List executorResources, String extraConfig) { String configUri; @@ -500,11 +525,13 @@ private ExecutorInfo.Builder getExecutorInfoBuilder(TopologyDetails details, Str .setName(executorName) .setExecutorId(ExecutorID.newBuilder().setValue(details.getId())) .setData(ByteString.copyFromUtf8(executorDataStr)) - .addAllResources(executorCpuResources) - .addAllResources(executorMemResources) - .addAllResources(executorPortsResources); + .addAllResources(executorResources); ICommandLineShim commandLineShim = CommandLineShimFactory.makeCommandLineShim(_container.isPresent(), extraConfig); + /** + * _container.isPresent() might be slightly misleading at first blush. It is only checking whether or not + * CONF_MESOS_CONTAINER_DOCKER_IMAGE is set to a value other than null. + */ if (_container.isPresent()) { executorInfoBuilder.setCommand(CommandInfo.newBuilder() .addUris(URI.newBuilder().setValue(configUri)) @@ -527,6 +554,7 @@ private ExecutorInfo.Builder getExecutorInfoBuilder(TopologyDetails details, Str return executorInfoBuilder; } + public Map> getTasksToLaunch(Topologies topologies, Map> slots, Map offerResourcesPerNode) { @@ -541,10 +569,11 @@ public Map> getTasksToLaunch(Topologies topologies, double workerMem = MesosCommon.topologyWorkerMem(mesosStormConf, topologyDetails); double executorCpu = MesosCommon.executorCpu(mesosStormConf); double executorMem = MesosCommon.executorMem(mesosStormConf); - double requiredCpu = workerCpu; - double requiredMem = workerMem; for (WorkerSlot slot : slotList) { + double requiredCpu = workerCpu; + double requiredMem = workerMem; + String workerHost = slot.getNodeId(); Long workerPort = Long.valueOf(slot.getPort()); @@ -557,7 +586,7 @@ public Map> getTasksToLaunch(Topologies topologies, workerPrefix = MesosCommon.getWorkerPrefix(mesosStormConf, topologyDetails); } - if (hostsUsedSoFar.contains(workerHost)) { + if (!hostsUsedSoFar.contains(workerHost)) { requiredCpu += executorCpu; requiredMem += executorMem; } @@ -578,13 +607,8 @@ public Map> getTasksToLaunch(Topologies topologies, continue; } - List executorCpuResources = new ArrayList<>(); - List executorMemResources = new ArrayList<>(); - List executorPortResources = new ArrayList<>(); - List workerCpuResources = new ArrayList<>(); - List workerMemResources = new ArrayList<>(); - List workerPortResources = new ArrayList<>(); - + List executorResources = new ArrayList<>(); + List workerResources = new ArrayList<>(); try { List scalarResourceEntryList = null; @@ -592,51 +616,33 @@ public Map> getTasksToLaunch(Topologies topologies, if (subtractExecutorResources) { scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.CPU, new ScalarResourceEntry(executorCpu)); - executorCpuResources.addAll(createMesosScalarResource(ResourceType.CPU, scalarResourceEntryList)); + executorResources.addAll(createMesosScalarResourceList(ResourceType.CPU, scalarResourceEntryList)); scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.MEM, new ScalarResourceEntry(executorMem)); - executorMemResources.addAll(createMesosScalarResource(ResourceType.MEM, scalarResourceEntryList)); + executorResources.addAll(createMesosScalarResourceList(ResourceType.MEM, scalarResourceEntryList)); } else { - // TODO(ksoundararaj): Do we need to ensure consistent roles for all executors belonging to same topology? - // If so, how do we do that? - executorCpuResources.add(createMesosScalarResource(ResourceType.CPU, new ScalarResourceEntry(executorCpu))); - executorMemResources.add(createMesosScalarResource(ResourceType.MEM, new ScalarResourceEntry(executorMem))); + executorResources.add(createMesosScalarResource(ResourceType.CPU, new ScalarResourceEntry(executorCpu))); + executorResources.add(createMesosScalarResource(ResourceType.MEM, new ScalarResourceEntry(executorMem))); } - /** - * Autostart logviewer cannot be supported. If we try to fetch a random port P - * from the list of available ports and assign it to the logviewer, its possible that - * one of the workers is already assigned to that port. So I am not supporting auto starting - * logviewer at this point - * - if (MesosCommon.autoStartLogViewer(mesosStormConf)) { - Long logViewerPort = offerResources.reserveRangeResource(ResourceType.PORTS, 1).get(0); - List logViewerPortList = new ArrayList<>(); - logViewerPortList.add(new RangeResourceEntry(logViewerPort, logViewerPort)); - executorPortResources.add(createMesosRangeResource(ResourceType.PORTS, logViewerPortList)); - } */ - - String supervisorStormLocalDir = (String) mesosStormConf.get(CONF_MESOS_SUPERVISOR_STORM_LOCAL_DIR); - if (supervisorStormLocalDir != null) { - extraConfig += String.format(" -c storm.local.dir=%s", supervisorStormLocalDir); - } + String supervisorStormLocalDir = getStormLocalDirForWorkers(); + extraConfig += String.format(" -c storm.local.dir=%s", supervisorStormLocalDir); scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.CPU, new ScalarResourceEntry(workerCpu)); - workerCpuResources.addAll(createMesosScalarResource(ResourceType.CPU, scalarResourceEntryList)); + workerResources.addAll(createMesosScalarResourceList(ResourceType.CPU, scalarResourceEntryList)); scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.MEM, new ScalarResourceEntry(workerMem)); - workerCpuResources.addAll(createMesosScalarResource(ResourceType.MEM, scalarResourceEntryList)); + workerResources.addAll(createMesosScalarResourceList(ResourceType.MEM, scalarResourceEntryList)); rangeResourceEntryList = offerResources.reserveAndGet(ResourceType.PORTS, new RangeResourceEntry(workerPort, workerPort)); for (ResourceEntry resourceEntry : rangeResourceEntryList) { - workerCpuResources.add(createMesosRangeResource(ResourceType.PORTS, (RangeResourceEntry) resourceEntry)); + workerResources.add(createMesosRangeResource(ResourceType.PORTS, (RangeResourceEntry) resourceEntry)); } - } catch (ResourceNotAvailabeException rexp) { + } catch (ResourceNotAvailableException rexp) { LOG.warn("Unable to launch worker %s. Required cpu: %f, Required mem: %f. Available OfferResources : %s", workerHost, requiredCpu, requiredMem, offerResources); continue; } - ExecutorInfo.Builder executorInfoBuilder = getExecutorInfoBuilder(topologyDetails, executorDataStr, executorName, executorCpuResources, - executorMemResources, executorPortResources, extraConfig); + ExecutorInfo.Builder executorInfoBuilder = getExecutorInfoBuilder(topologyDetails, executorDataStr, executorName, executorResources, extraConfig); TaskID taskId = TaskID.newBuilder() .setValue(MesosCommon.taskId(slot.getNodeId(), slot.getPort())) .build(); @@ -646,9 +652,7 @@ public Map> getTasksToLaunch(Topologies topologies, .setName(taskName) .setSlaveId(offerResources.getSlaveID()) .setExecutor(executorInfoBuilder.build()) - .addAllResources(workerCpuResources) - .addAllResources(workerMemResources) - .addAllResources(workerPortResources) + .addAllResources(workerResources) .build(); List taskInfoList = tasksToLaunchPerNode.get(slot.getNodeId()); diff --git a/storm/src/main/storm/mesos/resources/OfferResources.java b/storm/src/main/storm/mesos/resources/OfferResources.java index 76c8faa73..9318087d8 100644 --- a/storm/src/main/storm/mesos/resources/OfferResources.java +++ b/storm/src/main/storm/mesos/resources/OfferResources.java @@ -55,9 +55,7 @@ public OfferResources(String hostName) { public OfferResources(Protos.Offer offer) { initializeAvailableResources(); - if (slaveID == null) { - this.slaveID = offer.getSlaveId(); - } + this.slaveID = offer.getSlaveId(); this.hostName = offer.getHostname(); add(offer); } @@ -70,15 +68,15 @@ public void add(Protos.Offer offer) { offerList.add(offer); for (Protos.Resource r : offer.getResourcesList()) { + ResourceType resourceType = ResourceType.of(r.getName()); + ReservationType reservationType = (r.getRole().equals("*")) ? + ReservationType.UNRESERVED : ReservationType.STATICALLY_RESERVED; + if (r.hasReservation()) { // skip resources with dynamic reservations continue; } - ResourceType resourceType = ResourceType.of(r.getName()); - - ReservationType reservationType = (r.getRole().equals("*")) ? ReservationType.UNRESERVED : ReservationType.STATICALLY_RESERVED; - switch (resourceType) { case CPU: case MEM: @@ -115,22 +113,23 @@ public List getAllAvailableResources(ResourceType r return availableResources.get(resourceType).getAllAvailableResources(reservationType); } - public void reserve(ResourceType resourceType, ResourceEntry resource) throws ResourceNotAvailabeException { + public void reserve(ResourceType resourceType, ResourceEntry resource) throws ResourceNotAvailableException { if (availableResources.get(resourceType).isAvailable(resource)) { availableResources.get(resourceType).removeAndGet(resource); } } - public List reserveAndGet(ResourceType resourceType, ResourceEntry resource) throws ResourceNotAvailabeException { + public List reserveAndGet(ResourceType resourceType, ResourceEntry resource) throws ResourceNotAvailableException { if (availableResources.get(resourceType).isAvailable(resource)) { return availableResources.get(resourceType).removeAndGet(resource); } return new ArrayList<>(); } - public List reserveAndGet(ResourceType resourceType, ReservationType reservationType, ResourceEntry resource) throws ResourceNotAvailabeException { + public List reserveAndGet(ResourceType resourceType, ReservationType reservationType, ResourceEntry resource) throws + ResourceNotAvailableException { if (availableResources.get(resourceType).isAvailable(resource, reservationType)) { - availableResources.get(resourceType).removeAndGet(resource, reservationType); + return availableResources.get(resourceType).removeAndGet(resource, reservationType); } return new ArrayList<>(); } @@ -168,7 +167,7 @@ public boolean isFit(Map mesosStormConf, TopologyDetails topologyDetails, boolea return (isAvaliable(ResourceType.CPU, new ResourceEntries.ScalarResourceEntry(requestedWorkerCpu)) && isAvaliable(ResourceType.MEM, new ResourceEntries.ScalarResourceEntry(requestedWorkerMem)) && - !this.getAllAvailableResources(ResourceType.PORTS).isEmpty()); + !getAllAvailableResources(ResourceType.PORTS).isEmpty()); } public boolean isFit(Map mesosStormConf, TopologyDetails topologyDetails, Long port, boolean supervisorExists) { diff --git a/storm/src/main/storm/mesos/resources/RangeResource.java b/storm/src/main/storm/mesos/resources/RangeResource.java index 577d6112d..58d0b9906 100644 --- a/storm/src/main/storm/mesos/resources/RangeResource.java +++ b/storm/src/main/storm/mesos/resources/RangeResource.java @@ -94,14 +94,14 @@ public void add(RangeResourceEntry rangeResourceEntry, ReservationType reservati * {@param rangeResourceEntry} range resource to removeAndGet */ @Override - public List removeAndGet(RangeResourceEntry rangeResourceEntry) throws ResourceNotAvailabeException { + public List removeAndGet(RangeResourceEntry rangeResourceEntry) throws ResourceNotAvailableException { if (isAvailable(rangeResourceEntry)) { return removeAndGet(availableResourcesByReservationType.keySet(), rangeResourceEntry); } String message = String.format("ResourceType '%s' is not available. Requested value: %s Available: %s", resourceType, rangeResourceEntry, toString()); - throw new ResourceNotAvailabeException(message); + throw new ResourceNotAvailableException(message); } @@ -109,11 +109,11 @@ public List removeAndGet(RangeResourceEntry rangeResourceEntry) t * Remove/Reserve range from available ranges. * {@param rangeResourceEntry} range resource to removeAndGet * {@parm reservationType} reservation type of resource that needs to be removed. If the resource represented by rangeResourceEntry - * of the reservation type specied by this parameter is not available, then {@link storm.mesos.resources.ResourceNotAvailabeException} + * of the reservation type specied by this parameter is not available, then {@link ResourceNotAvailableException} * is thrown */ @Override - public List removeAndGet(RangeResourceEntry rangeResourceEntry, ReservationType reservationType) throws ResourceNotAvailabeException { + public List removeAndGet(RangeResourceEntry rangeResourceEntry, ReservationType reservationType) throws ResourceNotAvailableException { if (isAvailable(rangeResourceEntry, reservationType)) { List reservationTypeList = new ArrayList<>(); @@ -123,7 +123,7 @@ public List removeAndGet(RangeResourceEntry rangeResourceEntry, R String message = String.format("ResourceType '%s' of reservationType '%s' is not available. Requested value: %s Available: %s", resourceType, reservationType.toString(), rangeResourceEntry.toString(), toString(availableResourcesByReservationType.get(reservationType))); - throw new ResourceNotAvailabeException(message); + throw new ResourceNotAvailableException(message); } /** @@ -134,7 +134,8 @@ public List removeAndGet(RangeResourceEntry rangeResourceEntry, R * are available, the priority of reservation type removed is governed by {@link storm.mesos.resources.DefaultReservationTypeComparator} */ @Override - public List removeAndGet(RangeResourceEntry rangeResourceEntry, Comparator reservationTypeCompartor) throws ResourceNotAvailabeException { + public List removeAndGet(RangeResourceEntry rangeResourceEntry, Comparator reservationTypeCompartor) throws + ResourceNotAvailableException { if (isAvailable(rangeResourceEntry)) { List reservationTypeList = Arrays.asList(ReservationType.values()); Collections.sort(reservationTypeList, reservationTypeCompartor); @@ -143,7 +144,40 @@ public List removeAndGet(RangeResourceEntry rangeResourceEntry, C String message = String.format("ResourceType '%s' is not available. Requested value: %s Available: %s", resourceType, rangeResourceEntry, toString()); - throw new ResourceNotAvailabeException(message); + throw new ResourceNotAvailableException(message); + } + + private List removeAndGet(Collection reservationTypes, RangeResourceEntry desiredRange) { + List removedResources = new ArrayList<>(); + Long desiredBegin = desiredRange.getBegin(); + Long desiredEnd = desiredRange.getEnd(); + + for (ReservationType reservationType : reservationTypes) { + List availableRanges = availableResourcesByReservationType.get(reservationType); + for (int i = 0; i < availableRanges.size(); i++) { + RangeResourceEntry availableRange = availableRanges.get(i); + if (desiredBegin >= availableRange.getBegin() && desiredEnd <= availableRange.getEnd()) { + availableRanges.remove(i); + // We already removed the entry. So when beginValue == endValue, + // we dont have to add a new entry + if (availableRange.getBegin().equals(availableRange.getEnd()) || (availableRange.getBegin().equals(desiredBegin) && availableRange.getEnd().equals(desiredEnd))) { + removedResources.add(availableRange); + return removedResources; + } else if (desiredBegin > availableRange.getBegin() && availableRange.getEnd().equals(desiredEnd)) { + availableRanges.add(new RangeResourceEntry(reservationType, availableRange.getBegin(), desiredBegin - 1)); + removedResources.add(new RangeResourceEntry(reservationType, desiredBegin, desiredEnd)); + } else if (availableRange.getBegin().equals(desiredBegin) && desiredEnd < availableRange.getEnd()) { + availableRanges.add(new RangeResourceEntry(reservationType, desiredEnd + 1, availableRange.getEnd())); + removedResources.add(new RangeResourceEntry(reservationType, desiredBegin, desiredEnd)); + } else if (desiredBegin > availableRange.getBegin() && desiredEnd < availableRange.getEnd()) { + availableRanges.add(new RangeResourceEntry(reservationType, availableRange.getBegin(), desiredBegin - 1)); + availableRanges.add(new RangeResourceEntry(reservationType, desiredEnd + 1, availableRange.getEnd())); + removedResources.add(new RangeResourceEntry(reservationType, desiredBegin, desiredEnd)); + } + } + } + } + return removedResources; } public String toString(List ranges) { @@ -176,39 +210,4 @@ public String toString() { } return toString(resourceRanges); } - - private List removeAndGet(Collection reservationTypes, RangeResourceEntry rangeResourceEntry) { - List removedResources = new ArrayList<>(); - Long beginValue = rangeResourceEntry.getBegin(); - Long endValue = rangeResourceEntry.getEnd(); - - for (ReservationType reservationType : reservationTypes) { - List rangeResourceEntryList = availableResourcesByReservationType.get(reservationType); - for (int i = 0; i < rangeResourceEntryList.size(); i++) { - RangeResourceEntry tmp = rangeResourceEntryList.get(i); - if (beginValue >= tmp.getBegin() && endValue <= tmp.getEnd()) { - rangeResourceEntryList.remove(i); - // We already removed the entry. So when beginValue == endValue, - // we dont have to add a new entry - if (tmp.getBegin().equals(tmp.getEnd()) || (tmp.getBegin().equals(beginValue) && tmp.getEnd().equals(endValue))) { - removedResources.add(tmp); - return removedResources; - } - - if (beginValue > tmp.getBegin() && tmp.getEnd().equals(endValue)) { - rangeResourceEntryList.add(new RangeResourceEntry(reservationType, tmp.getBegin(), beginValue - 1)); - removedResources.add(new RangeResourceEntry(reservationType, beginValue, endValue)); - } else if (tmp.getBegin().equals(beginValue) && endValue < tmp.getEnd()) { - rangeResourceEntryList.add(new RangeResourceEntry(reservationType, endValue + 1, tmp.getEnd())); - removedResources.add(new RangeResourceEntry(reservationType, beginValue, endValue)); - } else if (beginValue > tmp.getBegin() && endValue < tmp.getEnd()) { - rangeResourceEntryList.add(new RangeResourceEntry(reservationType, tmp.getBegin(), beginValue - 1)); - rangeResourceEntryList.add(new RangeResourceEntry(reservationType, endValue + 1, tmp.getEnd())); - removedResources.add(new RangeResourceEntry(reservationType, beginValue, endValue)); - } - } - } - } - return removedResources; - } } diff --git a/storm/src/main/storm/mesos/resources/Resource.java b/storm/src/main/storm/mesos/resources/Resource.java index 67346dace..2e5931570 100644 --- a/storm/src/main/storm/mesos/resources/Resource.java +++ b/storm/src/main/storm/mesos/resources/Resource.java @@ -36,10 +36,10 @@ public interface Resource> { public void add(T resourceEntry, ReservationType reservationType); - public List removeAndGet(T resourceEntry) throws ResourceNotAvailabeException; + public List removeAndGet(T resourceEntry) throws ResourceNotAvailableException; - public List removeAndGet(T value, ReservationType reservationType) throws ResourceNotAvailabeException; + public List removeAndGet(T value, ReservationType reservationType) throws ResourceNotAvailableException; - public List removeAndGet(T value, Comparator reservationTypeComparator) throws ResourceNotAvailabeException; + public List removeAndGet(T value, Comparator reservationTypeComparator) throws ResourceNotAvailableException; } diff --git a/storm/src/main/storm/mesos/resources/ResourceEntries.java b/storm/src/main/storm/mesos/resources/ResourceEntries.java index 848ff8edc..9ec3a0c23 100644 --- a/storm/src/main/storm/mesos/resources/ResourceEntries.java +++ b/storm/src/main/storm/mesos/resources/ResourceEntries.java @@ -60,7 +60,6 @@ public static final class RangeResourceEntry implements ResourceEntry { private Long end; public RangeResourceEntry(Long begin, Long end) { - this.reservationType = reservationType; this.begin = begin; this.end = end; } @@ -83,9 +82,21 @@ public ReservationType getReservationType() { return reservationType; } - @Override + /** + * Lets say, we have a range [u,v]. Using this add function, we can expand the range to [w,x] if and + * only if one of the following conditions are satisfied + * `w < u` + * `x > v` + * `w < u` and `x > v` + * In case of a disjoint (u,v) and (w,x), no action is taken. + */ public RangeResourceEntry add(ResourceEntry resourceEntry) { RangeResourceEntry rangeResourceEntry = (RangeResourceEntry) resourceEntry; + + if (rangeResourceEntry.getBegin() > this.end || rangeResourceEntry.getEnd() < this.begin) { + return this; + } + if (this.begin < rangeResourceEntry.getBegin()) { this.begin = rangeResourceEntry.getBegin(); } @@ -95,14 +106,13 @@ public RangeResourceEntry add(ResourceEntry resourceEntry) { return this; } - @Override public RangeResourceEntry remove(ResourceEntry resourceEntry) { RangeResourceEntry rangeResourceEntry = (RangeResourceEntry) resourceEntry; - if (this.begin > rangeResourceEntry.getBegin()) { + if (this.begin < rangeResourceEntry.getBegin()) { this.begin = rangeResourceEntry.getBegin(); } - if (this.end < rangeResourceEntry.getEnd()) { + if (this.end > rangeResourceEntry.getEnd()) { this.end = rangeResourceEntry.getEnd(); } return this; diff --git a/storm/src/main/storm/mesos/resources/ResourceNotAvailabeException.java b/storm/src/main/storm/mesos/resources/ResourceNotAvailableException.java similarity index 87% rename from storm/src/main/storm/mesos/resources/ResourceNotAvailabeException.java rename to storm/src/main/storm/mesos/resources/ResourceNotAvailableException.java index 7d0155237..d82f0e2ff 100644 --- a/storm/src/main/storm/mesos/resources/ResourceNotAvailabeException.java +++ b/storm/src/main/storm/mesos/resources/ResourceNotAvailableException.java @@ -17,8 +17,8 @@ */ package storm.mesos.resources; -public class ResourceNotAvailabeException extends Exception { - public ResourceNotAvailabeException(String message) { +public class ResourceNotAvailableException extends Exception { + public ResourceNotAvailableException(String message) { super(message); } } diff --git a/storm/src/main/storm/mesos/resources/ResourceType.java b/storm/src/main/storm/mesos/resources/ResourceType.java index f8a0f4fc3..f13705e48 100644 --- a/storm/src/main/storm/mesos/resources/ResourceType.java +++ b/storm/src/main/storm/mesos/resources/ResourceType.java @@ -32,17 +32,17 @@ public enum ResourceType { private final String resourceType; private static final Logger LOG = LoggerFactory.getLogger(ResourceType.class); - private static final Map map = new HashMap<>(ResourceType.values().length); + private static final Map availableResourceTypes = new HashMap<>(ResourceType.values().length); static { for (ResourceType r : ResourceType.values()) { - map.put(r.resourceType, r); + availableResourceTypes.put(r.resourceType, r); } } public static ResourceType of(String name) { - ResourceType result = map.get(name); + ResourceType result = availableResourceTypes.get(name); if (result == null) { LOG.warn("Unexpected resource type {}", name); } diff --git a/storm/src/main/storm/mesos/resources/ScalarResource.java b/storm/src/main/storm/mesos/resources/ScalarResource.java index 5773bf0c9..4a989e9e2 100644 --- a/storm/src/main/storm/mesos/resources/ScalarResource.java +++ b/storm/src/main/storm/mesos/resources/ScalarResource.java @@ -50,7 +50,7 @@ public boolean isAvailable(ScalarResourceEntry scalarResourceEntry) { } public boolean isAvailable(ScalarResourceEntry scalarResourceEntry, ReservationType reservationType) { - return (availableResourcesByReservationType.get(reservationType).getValue() <= scalarResourceEntry.getValue()); + return (availableResourcesByReservationType.get(reservationType).getValue() >= scalarResourceEntry.getValue()); } public Double getTotalAvailableResource(ReservationType reservationType) { @@ -79,13 +79,13 @@ public void add(ScalarResourceEntry scalarResourceEntry, ReservationType reserva totalAvailableResource += scalarResourceEntry.getValue(); } - public List removeAndGet(ScalarResourceEntry scalarResourceEntry) throws ResourceNotAvailabeException { + public List removeAndGet(ScalarResourceEntry scalarResourceEntry) throws ResourceNotAvailableException { return removeAndGet(scalarResourceEntry, availableResourcesByReservationType.keySet()); } - public List reserveScalarResource(ResourceType resourceType, ScalarResourceEntry requiredValue) throws ResourceNotAvailabeException { - if (totalAvailableResource <= requiredValue.getValue()) { - throw new ResourceNotAvailabeException(String.format("resourceType: {} is not available. Requested {} Available {}", + public List reserveScalarResource(ResourceType resourceType, ScalarResourceEntry requiredValue) throws ResourceNotAvailableException { + if (totalAvailableResource < requiredValue.getValue()) { + throw new ResourceNotAvailableException(String.format("resourceType: {} is not available. Requested {} Available {}", resourceType, requiredValue, totalAvailableResource)); } return removeAndGet(requiredValue); @@ -98,7 +98,8 @@ public List reserveScalarResource(ResourceType resourceType, Scal * to determine the priority of the reservation type. When resources of all reservations are available, resources * are removed in the priority order specified by this comparator. */ - public List removeAndGet(ScalarResourceEntry scalarResourceEntry, Comparator reservationTypeComparator) throws ResourceNotAvailabeException { + public List removeAndGet(ScalarResourceEntry scalarResourceEntry, Comparator reservationTypeComparator) throws + ResourceNotAvailableException { List reservationTypeList = Arrays.asList(ReservationType.values()); Collections.sort(reservationTypeList, reservationTypeComparator); return removeAndGet(scalarResourceEntry, reservationTypeList); @@ -110,11 +111,11 @@ public List removeAndGet(ScalarResourceEntry scalarResourceEntry, * {@param scalarResourceEntry} amount of scalar resource to removeAndGet/decrement. * {@link storm.mesos.resources.DefaultReservationTypeComparator} determines the priority of the reservation type. */ - public List removeAndGet(ScalarResourceEntry scalarResourceEntry, ReservationType reservationType) throws ResourceNotAvailabeException { + public List removeAndGet(ScalarResourceEntry scalarResourceEntry, ReservationType reservationType) throws ResourceNotAvailableException { ScalarResourceEntry availableResource = availableResourcesByReservationType.get(reservationType); List reservedResources = new ArrayList<>(); - if (scalarResourceEntry.getValue() < availableResource.getValue()) { + if (scalarResourceEntry.getValue() <= availableResource.getValue()) { availableResourcesByReservationType.put(reservationType, availableResource.remove(scalarResourceEntry)); totalAvailableResource -= scalarResourceEntry.getValue(); reservedResources.add(new ScalarResourceEntry(scalarResourceEntry.getReservationType(), scalarResourceEntry.getValue())); @@ -122,7 +123,7 @@ public List removeAndGet(ScalarResourceEntry scalarResourceEntry, } String message = String.format("ResourceType '%s' of reservationType '%s' is not available. Requested value: %s Available: %s", resourceType, reservationType.toString(), scalarResourceEntry.getValue(), availableResourcesByReservationType.get(reservationType)); - throw new ResourceNotAvailabeException(message); + throw new ResourceNotAvailableException(message); } public String toString() { @@ -134,14 +135,15 @@ public String toString() { return String.format("Resource %s - Total available : %f Total available by reservation type : [ %s ]", resourceType.toString(), totalAvailableResource, tmp); } - private List removeAndGet(ScalarResourceEntry scalarResourceEntry, Collection reservationTypesListByPriority) throws ResourceNotAvailabeException { + private List removeAndGet(ScalarResourceEntry scalarResourceEntry, Collection reservationTypesListByPriority) throws + ResourceNotAvailableException { Double requiredValue = scalarResourceEntry.getValue(); List reservedResources = new ArrayList<>(); if (requiredValue > totalAvailableResource) { String message = String.format("ResourceType '%s' is not available. Requested value: %s Available: %s", resourceType, requiredValue, totalAvailableResource); - throw new ResourceNotAvailabeException(message); + throw new ResourceNotAvailableException(message); } for (ReservationType reservationType : reservationTypesListByPriority) { @@ -154,6 +156,8 @@ private List removeAndGet(ScalarResourceEntry scalarResourceEntry reservedResources.add(new ScalarResourceEntry(reservationType, requiredValue)); return reservedResources; } else if (availableResourceValue > 0) { + // Fact that we are here => 0 < availableResourceValue < requiredValue. + // So we could entire availableResourceValue. availableResourcesByReservationType.put(reservationType, new ScalarResourceEntry(reservationType, 0.0)); requiredValue -= availableResourceValue; totalAvailableResource -= availableResourceValue; diff --git a/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java b/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java index bfd35a195..fc1d4d33b 100644 --- a/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java +++ b/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java @@ -28,7 +28,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import storm.mesos.resources.OfferResources; -import storm.mesos.resources.ResourceNotAvailabeException; +import storm.mesos.resources.ResourceNotAvailableException; import storm.mesos.util.MesosCommon; import storm.mesos.util.RotatingMap; @@ -63,10 +63,10 @@ private List getMesosWorkerSlots(Map of double requestedWorkerMem = MesosCommon.topologyWorkerMem(mesosStormConf, topologyDetails); List mesosWorkerSlots = new ArrayList<>(); - boolean slotFound = true; + boolean slotFound = false; int slotsNeeded = topologyDetails.getNumWorkers(); - while (slotFound && slotsNeeded > 0) { + do { slotFound = false; for (String currentNode : offerResourcesPerNode.keySet()) { OfferResources offerResources = offerResourcesPerNode.get(currentNode); @@ -81,23 +81,22 @@ private List getMesosWorkerSlots(Map of log.info("{} is a fit for {} requestedWorkerCpu: {} requestedWorkerMem: {}", offerResources.toString(), topologyDetails.getId(), requestedWorkerCpu, requestedWorkerMem); - nodesWithExistingSupervisors.add(currentNode); MesosWorkerSlot mesosWorkerSlot; - try { mesosWorkerSlot = SchedulerUtils.createMesosWorkerSlot(mesosStormConf, offerResources, topologyDetails, supervisorExists); - } catch (ResourceNotAvailabeException rexp) { + } catch (ResourceNotAvailableException rexp) { log.warn(rexp.getMessage()); continue; } + nodesWithExistingSupervisors.add(currentNode); mesosWorkerSlots.add(mesosWorkerSlot); slotFound = true; if (--slotsNeeded == 0) { break; } } - } + } while (slotFound && slotsNeeded > 0); return mesosWorkerSlots; } @@ -137,7 +136,6 @@ public List allSlotsAvailableForScheduling(RotatingMap getPorts(OfferResources offerResources, i public static MesosWorkerSlot createMesosWorkerSlot(Map mesosStormConf, OfferResources offerResources, TopologyDetails topologyDetails, - boolean supervisorExists) throws ResourceNotAvailabeException { + boolean supervisorExists) throws ResourceNotAvailableException { double requestedWorkerCpu = MesosCommon.topologyWorkerCpu(mesosStormConf, topologyDetails); double requestedWorkerMem = MesosCommon.topologyWorkerMem(mesosStormConf, topologyDetails); @@ -75,7 +72,7 @@ public static MesosWorkerSlot createMesosWorkerSlot(Map mesosStormConf, List ports = getPorts(offerResources, 1); if (ports.isEmpty()) { - throw new ResourceNotAvailabeException("No ports available to create MesosWorkerSlot."); + throw new ResourceNotAvailableException("No ports available to create MesosWorkerSlot."); } offerResources.reserve(ResourceType.PORTS, ports.get(0)); @@ -102,23 +99,4 @@ public static boolean supervisorExists(String offerHost, Collection> getOfferResourcesListPerNode(RotatingMap offers) { - Map> offerResourcesListPerNode = new HashMap<>(); - - for (Protos.Offer offer : offers.values()) { - String hostName = offer.getHostname(); - - List offerResourcesListForCurrentHost = offerResourcesListPerNode.get(hostName); - OfferResources offerResources = new OfferResources(offer); - if (offerResourcesListForCurrentHost == null) { - offerResourcesListPerNode.put(hostName, new ArrayList()); - } - offerResourcesListPerNode.get(hostName).add(offerResources); - log.info("Available resources at {}: {}", hostName, offerResources.toString()); - } - return offerResourcesListPerNode; - } - - } diff --git a/storm/src/main/storm/mesos/util/MesosCommon.java b/storm/src/main/storm/mesos/util/MesosCommon.java index 5095fcc5d..efa853f37 100644 --- a/storm/src/main/storm/mesos/util/MesosCommon.java +++ b/storm/src/main/storm/mesos/util/MesosCommon.java @@ -61,12 +61,11 @@ public class MesosCommon { public static final String DEFAULT_WORKER_NAME_PREFIX_DELIMITER = "_"; public static final String DEFAULT_MESOS_COMPONENT_NAME_DELIMITER = " | "; - public static String getNimbusHost(Map mesosStormConf) throws UnknownHostException { Optional nimbusHostFromConfig = Optional.fromNullable((String) mesosStormConf.get(Config.NIMBUS_HOST)); Optional nimbusHostFromEnv = Optional.fromNullable(System.getenv("MESOS_NIMBUS_HOST")); - return nimbusHostFromConfig.or(nimbusHostFromEnv).or(nimbusHostFromConfig).or(InetAddress.getLocalHost().getCanonicalHostName()); + return nimbusHostFromConfig.or(nimbusHostFromEnv).or(InetAddress.getLocalHost().getCanonicalHostName()); } public static String hostFromAssignmentId(String assignmentId, String delimiter) { @@ -145,14 +144,17 @@ public static Map getConsolidatedOfferResourcesPerNode(R for (Protos.Offer offer : offers.values()) { String hostName = offer.getHostname(); - OfferResources offerResourcesForHost = offerResourcesPerNode.get(hostName); - if (offerResourcesForHost == null) { - offerResourcesForHost = new OfferResources(offer); - offerResourcesPerNode.put(hostName, offerResourcesForHost); + OfferResources offerResources = offerResourcesPerNode.get(hostName); + if (offerResources == null) { + offerResources = new OfferResources(offer); + offerResourcesPerNode.put(hostName, offerResources); } else { - offerResourcesForHost.add(offer); + offerResources.add(offer); } - LOG.info("Available resources at " + hostName + ": " + offerResourcesForHost.toString()); + } + + for (OfferResources offerResources : offerResourcesPerNode.values()) { + LOG.info("Available resources at {}: {}", offerResources.getHostName(), offerResources.toString()); } return offerResourcesPerNode; } diff --git a/storm/src/main/storm/mesos/util/PrettyProtobuf.java b/storm/src/main/storm/mesos/util/PrettyProtobuf.java index ebf3c4efa..a6518be75 100644 --- a/storm/src/main/storm/mesos/util/PrettyProtobuf.java +++ b/storm/src/main/storm/mesos/util/PrettyProtobuf.java @@ -243,6 +243,6 @@ public String apply(OfferID o) { public static String offerIDListToString(List offerIDList) { List offerIDsAsStrings = Lists.transform(offerIDList, offerIDToStringTransform); - return "[" + StringUtils.join(offerIDsAsStrings, ", ") + "]"; + return String.format("[ %s ]", StringUtils.join(offerIDsAsStrings, ", ")); } } diff --git a/storm/src/main/storm/mesos/util/RotatingMap.java b/storm/src/main/storm/mesos/util/RotatingMap.java index 399e29bc3..6d6a6ace3 100644 --- a/storm/src/main/storm/mesos/util/RotatingMap.java +++ b/storm/src/main/storm/mesos/util/RotatingMap.java @@ -31,7 +31,7 @@ * The algorithm used will take between expirationSecs and * expirationSecs * (1 + 1 / (numBuckets-1)) to actually expire the message. *

- * get, put, removeAndGet, containsKey, and size take O(numBuckets) time to run. + * get, put, remove, containsKey, and size take O(numBuckets) time to run. *

* The advantage of this design is that the expiration thread only locks the object * for O(1) time, meaning the object is essentially always available for gets/puts. diff --git a/storm/src/test/storm/mesos/MesosCommonTest.java b/storm/src/test/storm/mesos/MesosCommonTest.java index 645b5900f..d177a483f 100644 --- a/storm/src/test/storm/mesos/MesosCommonTest.java +++ b/storm/src/test/storm/mesos/MesosCommonTest.java @@ -39,7 +39,7 @@ public class MesosCommonTest { private Map conf; private TopologyDetails info; private String topologyName = "t_name"; - private static final double DELTA = 0.0001; + private static final double DELTA_FOR_DOUBLE_COMPARISON = 0.0001; private final MesosNimbus mesosNimbus; public MesosCommonTest() { @@ -214,7 +214,7 @@ public void testTopologyWorkerCpu() throws Exception { // Test default value double result = MesosCommon.topologyWorkerCpu(conf, info); double expectedResult = MesosCommon.DEFAULT_WORKER_CPU; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); // Test what happens when config is too small double cpuConfig = MesosCommon.MESOS_MIN_CPU; @@ -222,20 +222,20 @@ public void testTopologyWorkerCpu() throws Exception { conf.put(MesosCommon.WORKER_CPU_CONF, cpuConfig); result = MesosCommon.topologyWorkerCpu(conf, info); expectedResult = MesosCommon.DEFAULT_WORKER_CPU; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); // Test what happens when config is null conf.put(MesosCommon.WORKER_CPU_CONF, null); result = MesosCommon.topologyWorkerCpu(conf, info); expectedResult = MesosCommon.DEFAULT_WORKER_CPU; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); // Test explicit value conf.put(MesosCommon.WORKER_CPU_CONF, 1.5); info = new TopologyDetails("t2", conf, new StormTopology(), 2); result = MesosCommon.topologyWorkerCpu(conf, info); expectedResult = 1.5; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); // Test string passed in conf = new HashMap<>(); @@ -243,7 +243,7 @@ public void testTopologyWorkerCpu() throws Exception { info = new TopologyDetails("t3", conf, new StormTopology(), 1); result = MesosCommon.topologyWorkerCpu(conf, info); expectedResult = 1; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); // Test that this value is not overwritten by Topology config Map nimbusConf = new HashMap<>(); @@ -253,7 +253,7 @@ public void testTopologyWorkerCpu() throws Exception { info = new TopologyDetails("t4", conf, new StormTopology(), 1); result = MesosCommon.topologyWorkerCpu(nimbusConf, info); expectedResult = 1.5; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); } @Test @@ -261,7 +261,7 @@ public void testTopologyWorkerMem() throws Exception { // Test default value double result = MesosCommon.topologyWorkerMem(conf, info); double expectedResult = MesosCommon.DEFAULT_WORKER_MEM_MB; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); // Test what happens when config is too small double memConfig = MesosCommon.MESOS_MIN_MEM_MB; @@ -269,27 +269,27 @@ public void testTopologyWorkerMem() throws Exception { conf.put(MesosCommon.WORKER_MEM_CONF, memConfig); result = MesosCommon.topologyWorkerMem(conf, info); expectedResult = MesosCommon.DEFAULT_WORKER_MEM_MB; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); // Test what happens when config is null conf.put(MesosCommon.WORKER_MEM_CONF, null); result = MesosCommon.topologyWorkerMem(conf, info); expectedResult = MesosCommon.DEFAULT_WORKER_MEM_MB; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); // Test explicit value conf.put(MesosCommon.WORKER_MEM_CONF, 1200); info = new TopologyDetails("t2", conf, new StormTopology(), 2); result = MesosCommon.topologyWorkerMem(conf, info); expectedResult = 1200; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); // Test string passed in conf.put(MesosCommon.WORKER_MEM_CONF, "1200"); info = new TopologyDetails("t3", conf, new StormTopology(), 1); result = MesosCommon.topologyWorkerMem(conf, info); expectedResult = 1000; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); // Test that this value is overwritten by Topology config Map nimbusConf = new HashMap<>(); @@ -298,7 +298,7 @@ public void testTopologyWorkerMem() throws Exception { info = new TopologyDetails("t4", conf, new StormTopology(), 1); result = MesosCommon.topologyWorkerMem(nimbusConf, info); expectedResult = 150; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); } @Test @@ -306,13 +306,13 @@ public void testExecutorCpu() throws Exception { // Test default config double result = MesosCommon.executorCpu(conf); double expectedResult = MesosCommon.DEFAULT_EXECUTOR_CPU; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); // Test explicit value conf.put(MesosCommon.EXECUTOR_CPU_CONF, 2.0); result = MesosCommon.executorCpu(conf); expectedResult = 2.0; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); } @Test @@ -320,13 +320,13 @@ public void testExecutorMem() throws Exception { // Test default config double result = MesosCommon.executorMem(conf); double expectedResult = MesosCommon.DEFAULT_EXECUTOR_MEM_MB; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); // Test explicit value conf.put(MesosCommon.EXECUTOR_MEM_CONF, 100); result = MesosCommon.executorMem(conf); expectedResult = 100; - assertEquals(result, expectedResult, DELTA); + assertEquals(result, expectedResult, DELTA_FOR_DOUBLE_COMPARISON); } @Test @@ -346,14 +346,14 @@ public void getConsolidatedOfferResourcesPerNode() { Map offerResourcesMap = MesosCommon.getConsolidatedOfferResourcesPerNode(r); OfferResources offerResources = offerResourcesMap.get("h1"); - assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU) == 210); - assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM) == 3000); - assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU, ReservationType.STATICALLY_RESERVED) == 200); - assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM, ReservationType.STATICALLY_RESERVED) == 2000); - assertTrue(TestUtils.calculateAllAvailableRangeResources(offerResources, ResourceType.PORTS, ReservationType.STATICALLY_RESERVED).size() == 0); + assertEquals(210, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(3000, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(200, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU, ReservationType.STATICALLY_RESERVED), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(2000, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM, ReservationType.STATICALLY_RESERVED), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(0, TestUtils.calculateAllAvailableRangeResources(offerResources, ResourceType.PORTS, ReservationType.STATICALLY_RESERVED).size(), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); offerResources = offerResourcesMap.get("h2"); - assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU, ReservationType.STATICALLY_RESERVED) == 0); - assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM, ReservationType.STATICALLY_RESERVED) == 0); - assertTrue(TestUtils.calculateAllAvailableRangeResources(offerResources, ResourceType.PORTS, ReservationType.UNRESERVED).size() == 100); + assertEquals(0, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU, ReservationType.STATICALLY_RESERVED), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(0, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM, ReservationType.STATICALLY_RESERVED), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(100, TestUtils.calculateAllAvailableRangeResources(offerResources, ResourceType.PORTS, ReservationType.UNRESERVED).size(), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); } } diff --git a/storm/src/test/storm/mesos/MesosNimbusTest.java b/storm/src/test/storm/mesos/MesosNimbusTest.java index fde746a46..4d8e3fbc0 100644 --- a/storm/src/test/storm/mesos/MesosNimbusTest.java +++ b/storm/src/test/storm/mesos/MesosNimbusTest.java @@ -27,8 +27,6 @@ import org.junit.Test; import org.mockito.Mockito; import storm.mesos.resources.OfferResources; -import storm.mesos.resources.ReservationType; -import storm.mesos.resources.Resource; import storm.mesos.resources.ResourceType; import storm.mesos.util.MesosCommon; import storm.mesos.util.RotatingMap; @@ -36,10 +34,8 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -77,8 +73,11 @@ private boolean hasResources(String role, List resourceList, Do case PORTS: Protos.Value.Ranges ranges = resource.getRanges(); for (Protos.Value.Range range : ranges.getRangeList()) { - for (long i = 0; i < (range.getEnd() - range.getBegin() + 1); i++) { - actualPorts.add(range.getBegin() + i); + long endValue = range.getEnd(); + long beginValue = range.getBegin(); + while (endValue >= beginValue) { + actualPorts.add(beginValue); + ++beginValue; } } } @@ -113,8 +112,8 @@ private boolean hasResources(List resourceList, Double cpus, Do } } - boolean hasPorts = (expectedPorts.size() == actualPorts.size()) && expectedPorts.containsAll(actualPorts); - return actualCpu.equals(cpus) && actualMem.equals(mem) && hasPorts; + boolean hasExpectedPorts = (expectedPorts.size() == actualPorts.size()) && expectedPorts.containsAll(actualPorts); + return actualCpu.equals(cpus) && actualMem.equals(mem) && hasExpectedPorts; } private boolean hasResources(List resourceList, Double cpus, Double mem) { @@ -252,7 +251,8 @@ public void testGetTasksToLaunchForOneTopologyWithOneOffer() { Map> tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); - assertTrue((tasksToLaunch.size() == 1) && (tasksToLaunch.get("h1").size() == 1)); + assertTrue((tasksToLaunch.size() == 1)); + assertTrue((tasksToLaunch.get("h1").size() == 1)); // One offer with sufficient resources spread across reserved and unreserved resources offer = TestUtils.buildOfferWithReservationAndPorts("O-1", "h1", 0.75, 750, 0.75, 850, 3100, 3101); @@ -266,7 +266,8 @@ public void testGetTasksToLaunchForOneTopologyWithOneOffer() { tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); - assertTrue((tasksToLaunch.size() == 1) && (tasksToLaunch.get("h1").size() == 1)); + assertTrue((tasksToLaunch.size() == 1)); + assertTrue((tasksToLaunch.get("h1").size() == 1)); assertTrue(hasResources(FRAMEWORK_ROLE, tasksToLaunch.get("h1").get(0), 0.75 - MesosCommon.DEFAULT_EXECUTOR_CPU, 850 - MesosCommon.DEFAULT_EXECUTOR_MEM_MB)); assertTrue(hasResources("*", tasksToLaunch.get("h1").get(0), 0.35, 650.0)); assertTrue(hasCorrectExecutorResources(tasksToLaunch.get("h1"))); @@ -285,7 +286,8 @@ public void testGetTasksToLaunchForOneTopologyWithOneOffer() { tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); - assertTrue((tasksToLaunch.size() == 1) && (tasksToLaunch.get("h1").size() == 1)); + assertTrue((tasksToLaunch.size() == 1)); + assertTrue(tasksToLaunch.get("h1").size() == 1); assertTrue(hasResources(FRAMEWORK_ROLE, tasksToLaunch.get("h1").get(0), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB)); assertTrue(hasCorrectExecutorResources(tasksToLaunch.get("h1"))); assertEquals(TestUtils.calculateAllAvailableScalarResources(offerResourcesPerNode.get("h1"), ResourceType.CPU), 0.4f, 0.01f); @@ -352,7 +354,8 @@ public void testGetTasksToLaunchForOneTopologyWithMultipleOffersOnSameHost() { workerSlots.add(new WorkerSlot("h1", 3100)); workerSlotsMap.put("t1", workerSlots); Map> tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); - assertTrue((tasksToLaunch.size() == 1) && (tasksToLaunch.get("h1").size() == 1)); + assertTrue((tasksToLaunch.size() == 1)); + assertTrue((tasksToLaunch.get("h1").size() == 1)); List taskInfoList = tasksToLaunch.get("h1"); assertTrue(hasResources("*", taskInfoList.get(0), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3100l)); assertTrue(hasCorrectExecutorResources(taskInfoList)); diff --git a/storm/src/test/storm/mesos/resources/OfferResourcesTest.java b/storm/src/test/storm/mesos/resources/OfferResourcesTest.java index c45d3c630..ed873054d 100644 --- a/storm/src/test/storm/mesos/resources/OfferResourcesTest.java +++ b/storm/src/test/storm/mesos/resources/OfferResourcesTest.java @@ -23,14 +23,14 @@ import org.mockito.runners.MockitoJUnitRunner; import storm.mesos.TestUtils; -import java.util.ArrayList; import java.util.List; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import storm.mesos.TestUtils; @RunWith(MockitoJUnitRunner.class) public class OfferResourcesTest { + private static final double DELTA_FOR_DOUBLE_COMPARISON = 0.0001; @Test public void testToIgnoreDynamicResources() { @@ -42,23 +42,23 @@ public void testToIgnoreDynamicResources() { Offer offer = TestUtils.buildOffer("0-1", "h1", 0, 0); OfferResources offerResources = new OfferResources(offer); - assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU) == 0); - assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM) == 0); + assertEquals(0, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU), DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(0, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM), DELTA_FOR_DOUBLE_COMPARISON); assertTrue(offerResources.getHostName().equals(offer.getHostname())); assertTrue(offerResources.getSlaveID().equals(offer.getSlaveId())); offer = TestUtils.buildOfferWithReservation("offer1", "h1", 2, 1000, 6, 1000); offerResources = new OfferResources(offer); - assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU) == 8); - assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM) == 2000); + assertEquals(8, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU), DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(2000, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM), DELTA_FOR_DOUBLE_COMPARISON); assertTrue(offerResources.getHostName().equals(offer.getHostname())); assertTrue(offerResources.getSlaveID().equals(offer.getSlaveId())); offer = TestUtils.buildOfferWithPorts("offer1", "h1", 2.0, 2000, 3000, 3100); offerResources = new OfferResources(offer); - assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU) == 2.0); - assertTrue(TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM) == 2000); + assertEquals(2.0, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU), DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(2000, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM),DELTA_FOR_DOUBLE_COMPARISON); List rangeResources = TestUtils.calculateAllAvailableRangeResources(offerResources, ResourceType.PORTS); assertTrue(rangeResources.size() == 101); } diff --git a/storm/src/test/storm/mesos/schedulers/SchedulerUtilsTest.java b/storm/src/test/storm/mesos/schedulers/SchedulerUtilsTest.java index b3de8af35..7da87e62b 100644 --- a/storm/src/test/storm/mesos/schedulers/SchedulerUtilsTest.java +++ b/storm/src/test/storm/mesos/schedulers/SchedulerUtilsTest.java @@ -24,14 +24,11 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.runners.MockitoJUnitRunner; -import storm.mesos.resources.OfferResources; import storm.mesos.util.MesosCommon; import storm.mesos.util.RotatingMap; import java.util.ArrayList; import java.util.Collection; -import java.util.List; -import java.util.Map; import static org.junit.Assert.assertEquals; import static storm.mesos.TestUtils.buildOffer; @@ -63,35 +60,4 @@ public void testSupervisorExists() throws Exception { assertEquals(true, SchedulerUtils.supervisorExists(hostName, existingSupervisors, "test-topology1-65-1442255385")); assertEquals(false, SchedulerUtils.supervisorExists(hostName, existingSupervisors, "test-topology2-65-1442255385")); } - - @Test - public void testGetOfferResourcesListPerNode() { - String hostName = sampleHost; - - buildOfferAndUpdateRotatingMap("offer1", hostName, 0, 1000); - buildOfferAndUpdateRotatingMap("offer2", hostName, 10, 0); - buildOfferAndUpdateRotatingMap("offer3", hostName, 0, 100.01); - buildOfferAndUpdateRotatingMap("offer4", hostName, 1.001, 0); - buildOfferAndUpdateRotatingMap("offer5", hostName, 0, 0.001); - buildOfferAndUpdateRotatingMap("offer6", hostName, 0.001, 0.01); - - Map> offerResourcesMap = SchedulerUtils.getOfferResourcesListPerNode(rotatingMap); - assertEquals(offerResourcesMap.size(), 1); - - List offerResources = offerResourcesMap.get("host1.east"); - assertEquals(offerResources.size(), 6); - - hostName = "host1.west"; - buildOfferAndUpdateRotatingMap("offer7", hostName, 0, 1000); - buildOfferAndUpdateRotatingMap("offer8", hostName, 10, 0); - - offerResourcesMap = SchedulerUtils.getOfferResourcesListPerNode(rotatingMap); - assertEquals(offerResourcesMap.size(), 2); - - offerResources = offerResourcesMap.get("host1.east"); - assertEquals(offerResources.size(), 6); - - offerResources = offerResourcesMap.get("host1.west"); - assertEquals(offerResources.size(), 2); - } } From 45f23e2f92ff3c43ab3bfc4f40cb504aedbcd23b Mon Sep 17 00:00:00 2001 From: Karthick Duraisamy Soundararaj Date: Mon, 13 Jun 2016 14:34:31 +0530 Subject: [PATCH 06/12] Addressed the following comments: 1. Refactor MesosNimbus.getTasksToLaunch as per Erik's comment 2. Refactor SchedulerUtils.getPorts as per Jessica's comment --- storm/src/main/storm/mesos/MesosNimbus.java | 22 +++++++++---------- .../mesos/schedulers/SchedulerUtils.java | 14 ++++-------- 2 files changed, 14 insertions(+), 22 deletions(-) diff --git a/storm/src/main/storm/mesos/MesosNimbus.java b/storm/src/main/storm/mesos/MesosNimbus.java index 4601d7317..cbf468d0c 100644 --- a/storm/src/main/storm/mesos/MesosNimbus.java +++ b/storm/src/main/storm/mesos/MesosNimbus.java @@ -563,30 +563,27 @@ public Map> getTasksToLaunch(Topologies topologies, for (String topologyId : slots.keySet()) { Collection slotList = slots.get(topologyId); TopologyDetails topologyDetails = topologies.getById(topologyId); - Set hostsUsedSoFar = new HashSet<>(); + Set hostsWithSupervisors = new HashSet<>(); - double workerCpu = MesosCommon.topologyWorkerCpu(mesosStormConf, topologyDetails); - double workerMem = MesosCommon.topologyWorkerMem(mesosStormConf, topologyDetails); double executorCpu = MesosCommon.executorCpu(mesosStormConf); double executorMem = MesosCommon.executorMem(mesosStormConf); for (WorkerSlot slot : slotList) { + double workerCpu = MesosCommon.topologyWorkerCpu(mesosStormConf, topologyDetails); + double workerMem = MesosCommon.topologyWorkerMem(mesosStormConf, topologyDetails); double requiredCpu = workerCpu; double requiredMem = workerMem; String workerHost = slot.getNodeId(); Long workerPort = Long.valueOf(slot.getPort()); - Boolean subtractExecutorResources = !hostsUsedSoFar.contains(workerHost); - hostsUsedSoFar.add(workerHost); - OfferResources offerResources = offerResourcesPerNode.get(slot.getNodeId()); String workerPrefix = ""; if (mesosStormConf.containsKey(MesosCommon.WORKER_NAME_PREFIX)) { workerPrefix = MesosCommon.getWorkerPrefix(mesosStormConf, topologyDetails); } - if (!hostsUsedSoFar.contains(workerHost)) { + if (!hostsWithSupervisors.contains(workerHost)) { requiredCpu += executorCpu; requiredMem += executorMem; } @@ -601,7 +598,7 @@ public Map> getTasksToLaunch(Topologies topologies, String executorDataStr = JSONValue.toJSONString(executorData); String extraConfig = ""; - if (!offerResources.isFit(mesosStormConf, topologyDetails, workerPort, !subtractExecutorResources)) { + if (!offerResources.isFit(mesosStormConf, topologyDetails, workerPort, hostsWithSupervisors.contains(workerHost))) { LOG.error(String.format("Unable to launch worker %s. Required cpu: %f, Required mem: %f. Available OfferResources : %s", workerHost, requiredCpu, requiredMem, offerResources)); continue; @@ -614,14 +611,14 @@ public Map> getTasksToLaunch(Topologies topologies, List scalarResourceEntryList = null; List rangeResourceEntryList = null; - if (subtractExecutorResources) { + if (hostsWithSupervisors.contains(workerHost)) { + executorResources.add(createMesosScalarResource(ResourceType.CPU, new ScalarResourceEntry(executorCpu))); + executorResources.add(createMesosScalarResource(ResourceType.MEM, new ScalarResourceEntry(executorMem))); + } else { scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.CPU, new ScalarResourceEntry(executorCpu)); executorResources.addAll(createMesosScalarResourceList(ResourceType.CPU, scalarResourceEntryList)); scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.MEM, new ScalarResourceEntry(executorMem)); executorResources.addAll(createMesosScalarResourceList(ResourceType.MEM, scalarResourceEntryList)); - } else { - executorResources.add(createMesosScalarResource(ResourceType.CPU, new ScalarResourceEntry(executorCpu))); - executorResources.add(createMesosScalarResource(ResourceType.MEM, new ScalarResourceEntry(executorMem))); } String supervisorStormLocalDir = getStormLocalDirForWorkers(); @@ -641,6 +638,7 @@ public Map> getTasksToLaunch(Topologies topologies, continue; } + hostsWithSupervisors.add(workerHost); ExecutorInfo.Builder executorInfoBuilder = getExecutorInfoBuilder(topologyDetails, executorDataStr, executorName, executorResources, extraConfig); TaskID taskId = TaskID.newBuilder() diff --git a/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java b/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java index c0ccf63b4..4c61355a4 100644 --- a/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java +++ b/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java @@ -19,8 +19,6 @@ import backtype.storm.scheduler.SupervisorDetails; import backtype.storm.scheduler.TopologyDetails; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import storm.mesos.resources.OfferResources; import storm.mesos.resources.ResourceNotAvailableException; import storm.mesos.resources.ResourceType; @@ -36,8 +34,6 @@ public class SchedulerUtils { - private static final Logger log = LoggerFactory.getLogger(SchedulerUtils.class); - public static List getPorts(OfferResources offerResources, int requiredCount) { List retVal = new ArrayList<>(); List resourceEntryList = offerResources.getAllAvailableResources(ResourceType.PORTS); @@ -45,17 +41,15 @@ public static List getPorts(OfferResources offerResources, i for (RangeResourceEntry rangeResourceEntry : resourceEntryList) { Long begin = rangeResourceEntry.getBegin(); Long end = rangeResourceEntry.getEnd(); - for (int i = 0; i <= (end - begin) && requiredCount > 0; i++) { - retVal.add(new RangeResourceEntry(begin, begin + i)); - requiredCount--; + while (begin <= end && requiredCount > 0) { + retVal.add(new RangeResourceEntry(begin, begin)); + ++begin; + --requiredCount; } } - return retVal; } - - public static MesosWorkerSlot createMesosWorkerSlot(Map mesosStormConf, OfferResources offerResources, TopologyDetails topologyDetails, From be33c903665a10930c268d5907da094d5d6144ca Mon Sep 17 00:00:00 2001 From: Karthick Duraisamy Soundararaj Date: Mon, 13 Jun 2016 15:42:10 +0530 Subject: [PATCH 07/12] Missed a couple of comments on MesosNimbusTest --- storm/src/test/storm/mesos/MesosNimbusTest.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/storm/src/test/storm/mesos/MesosNimbusTest.java b/storm/src/test/storm/mesos/MesosNimbusTest.java index 4d8e3fbc0..2c3682f08 100644 --- a/storm/src/test/storm/mesos/MesosNimbusTest.java +++ b/storm/src/test/storm/mesos/MesosNimbusTest.java @@ -83,8 +83,8 @@ private boolean hasResources(String role, List resourceList, Do } } - boolean hasPorts = (expectedPorts.size() == actualPorts.size()) && expectedPorts.containsAll(actualPorts); - return actualCpu.equals(cpus) && actualMem.equals(mem) && hasPorts; + boolean hasExpectedPorts = (expectedPorts.size() == actualPorts.size()) && expectedPorts.containsAll(actualPorts); + return actualCpu.equals(cpus) && actualMem.equals(mem) && hasExpectedPorts; } private boolean hasResources(List resourceList, Double cpus, Double mem, Long port) { @@ -105,12 +105,15 @@ private boolean hasResources(List resourceList, Double cpus, Do case PORTS: Protos.Value.Ranges ranges = resource.getRanges(); for (Protos.Value.Range range : ranges.getRangeList()) { - for (long i = 0; i < (range.getEnd() - range.getBegin() + 1); i++) { - actualPorts.add(range.getBegin() + i); + long endValue = range.getEnd(); + long beginValue = range.getBegin(); + while (endValue >= beginValue) { + actualPorts.add(beginValue); + ++beginValue; } } + } } - } boolean hasExpectedPorts = (expectedPorts.size() == actualPorts.size()) && expectedPorts.containsAll(actualPorts); return actualCpu.equals(cpus) && actualMem.equals(mem) && hasExpectedPorts; From 170259de541448e104e3587f503612eeeb5272c1 Mon Sep 17 00:00:00 2001 From: Karthick Duraisamy Soundararaj Date: Thu, 16 Jun 2016 08:58:47 +0530 Subject: [PATCH 08/12] Addressing Erik's comments - Second round --- storm/src/main/storm/mesos/MesosNimbus.java | 8 ++++---- storm/src/main/storm/mesos/resources/OfferResources.java | 1 + storm/src/main/storm/mesos/resources/RangeResource.java | 6 +++--- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/storm/src/main/storm/mesos/MesosNimbus.java b/storm/src/main/storm/mesos/MesosNimbus.java index cbf468d0c..5c9a0ad6f 100644 --- a/storm/src/main/storm/mesos/MesosNimbus.java +++ b/storm/src/main/storm/mesos/MesosNimbus.java @@ -415,10 +415,10 @@ public void assignSlots(Topologies topologies, Map offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(_offers); Map> tasksToLaunchPerNode = getTasksToLaunch(topologies, slotsForTopologiesNeedingAssignments, offerResourcesPerNode); @@ -565,12 +565,12 @@ public Map> getTasksToLaunch(Topologies topologies, TopologyDetails topologyDetails = topologies.getById(topologyId); Set hostsWithSupervisors = new HashSet<>(); + double workerCpu = MesosCommon.topologyWorkerCpu(mesosStormConf, topologyDetails); + double workerMem = MesosCommon.topologyWorkerMem(mesosStormConf, topologyDetails); double executorCpu = MesosCommon.executorCpu(mesosStormConf); double executorMem = MesosCommon.executorMem(mesosStormConf); for (WorkerSlot slot : slotList) { - double workerCpu = MesosCommon.topologyWorkerCpu(mesosStormConf, topologyDetails); - double workerMem = MesosCommon.topologyWorkerMem(mesosStormConf, topologyDetails); double requiredCpu = workerCpu; double requiredMem = workerMem; diff --git a/storm/src/main/storm/mesos/resources/OfferResources.java b/storm/src/main/storm/mesos/resources/OfferResources.java index 9318087d8..4cafff737 100644 --- a/storm/src/main/storm/mesos/resources/OfferResources.java +++ b/storm/src/main/storm/mesos/resources/OfferResources.java @@ -90,6 +90,7 @@ public void add(Protos.Offer offer) { } break; case DISK: + // TODO: Support disk resource isolation (https://github.com/mesos/storm/issues/147) break; default: log.warn(String.format("Found unsupported resourceType '%s' while adding offer %s", resourceType, PrettyProtobuf.offerToString(offer))); diff --git a/storm/src/main/storm/mesos/resources/RangeResource.java b/storm/src/main/storm/mesos/resources/RangeResource.java index 58d0b9906..aa1b4e40f 100644 --- a/storm/src/main/storm/mesos/resources/RangeResource.java +++ b/storm/src/main/storm/mesos/resources/RangeResource.java @@ -108,8 +108,8 @@ public List removeAndGet(RangeResourceEntry rangeResourceEntry) t /** * Remove/Reserve range from available ranges. * {@param rangeResourceEntry} range resource to removeAndGet - * {@parm reservationType} reservation type of resource that needs to be removed. If the resource represented by rangeResourceEntry - * of the reservation type specied by this parameter is not available, then {@link ResourceNotAvailableException} + * {@param reservationType} reservation type of resource that needs to be removed. If the resource represented by rangeResourceEntry + * of the reservation type specified by this parameter is not available, then {@link ResourceNotAvailableException} * is thrown */ @Override @@ -159,7 +159,7 @@ private List removeAndGet(Collection reservation if (desiredBegin >= availableRange.getBegin() && desiredEnd <= availableRange.getEnd()) { availableRanges.remove(i); // We already removed the entry. So when beginValue == endValue, - // we dont have to add a new entry + // we don't have to add a new entry if (availableRange.getBegin().equals(availableRange.getEnd()) || (availableRange.getBegin().equals(desiredBegin) && availableRange.getEnd().equals(desiredEnd))) { removedResources.add(availableRange); return removedResources; From b65efff323afc61c9b2cd1507df389840ab0c54b Mon Sep 17 00:00:00 2001 From: Karthick Duraisamy Soundararaj Date: Thu, 16 Jun 2016 21:41:38 +0530 Subject: [PATCH 09/12] Addressing Jessica's comments --- .../storm/mesos/resources/RangeResource.java | 19 +++++++++---------- .../mesos/resources/ResourceEntries.java | 4 ++-- 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/storm/src/main/storm/mesos/resources/RangeResource.java b/storm/src/main/storm/mesos/resources/RangeResource.java index aa1b4e40f..21bc88586 100644 --- a/storm/src/main/storm/mesos/resources/RangeResource.java +++ b/storm/src/main/storm/mesos/resources/RangeResource.java @@ -158,22 +158,21 @@ private List removeAndGet(Collection reservation RangeResourceEntry availableRange = availableRanges.get(i); if (desiredBegin >= availableRange.getBegin() && desiredEnd <= availableRange.getEnd()) { availableRanges.remove(i); - // We already removed the entry. So when beginValue == endValue, - // we don't have to add a new entry + // If this is exactly the ranges we were looking for, then we can use them if (availableRange.getBegin().equals(availableRange.getEnd()) || (availableRange.getBegin().equals(desiredBegin) && availableRange.getEnd().equals(desiredEnd))) { removedResources.add(availableRange); return removedResources; - } else if (desiredBegin > availableRange.getBegin() && availableRange.getEnd().equals(desiredEnd)) { - availableRanges.add(new RangeResourceEntry(reservationType, availableRange.getBegin(), desiredBegin - 1)); - removedResources.add(new RangeResourceEntry(reservationType, desiredBegin, desiredEnd)); - } else if (availableRange.getBegin().equals(desiredBegin) && desiredEnd < availableRange.getEnd()) { - availableRanges.add(new RangeResourceEntry(reservationType, desiredEnd + 1, availableRange.getEnd())); - removedResources.add(new RangeResourceEntry(reservationType, desiredBegin, desiredEnd)); - } else if (desiredBegin > availableRange.getBegin() && desiredEnd < availableRange.getEnd()) { + } + // Salvage resources before the beginning of the requested range + if (availableRange.getBegin() < desiredBegin) { availableRanges.add(new RangeResourceEntry(reservationType, availableRange.getBegin(), desiredBegin - 1)); + } + // Salvage resources after the end of the requested range + if (availableRange.getEnd() > desiredEnd) { availableRanges.add(new RangeResourceEntry(reservationType, desiredEnd + 1, availableRange.getEnd())); - removedResources.add(new RangeResourceEntry(reservationType, desiredBegin, desiredEnd)); } + // Now that we've salvaged all available resources, add the resources for the specifically requested range + removedResources.add(new RangeResourceEntry(reservationType, desiredBegin, desiredEnd)); } } } diff --git a/storm/src/main/storm/mesos/resources/ResourceEntries.java b/storm/src/main/storm/mesos/resources/ResourceEntries.java index 9ec3a0c23..b1761dc69 100644 --- a/storm/src/main/storm/mesos/resources/ResourceEntries.java +++ b/storm/src/main/storm/mesos/resources/ResourceEntries.java @@ -97,10 +97,10 @@ public RangeResourceEntry add(ResourceEntry resourceEntry) { return this; } - if (this.begin < rangeResourceEntry.getBegin()) { + if (this.begin < rangeResourceEntry.getBegin() || this.begin == rangeResourceEntry.getEnd() + 1) { this.begin = rangeResourceEntry.getBegin(); } - if (this.end > rangeResourceEntry.getBegin()) { + if (this.end > rangeResourceEntry.getBegin() || this.end == rangeResourceEntry.getBegin() + 1) { this.end = rangeResourceEntry.getBegin(); } return this; From b817008a42576bc66e13a94ef1c47ddbfa38897a Mon Sep 17 00:00:00 2001 From: Karthick Duraisamy Soundararaj Date: Fri, 17 Jun 2016 19:25:40 +0530 Subject: [PATCH 10/12] Fixing range addition as per Jessica's comment --- .../mesos/resources/ResourceEntries.java | 8 +- .../mesos/resources/ResourceEntryTest.java | 137 ++++++++++++++++++ 2 files changed, 141 insertions(+), 4 deletions(-) create mode 100644 storm/src/test/storm/mesos/resources/ResourceEntryTest.java diff --git a/storm/src/main/storm/mesos/resources/ResourceEntries.java b/storm/src/main/storm/mesos/resources/ResourceEntries.java index b1761dc69..c66e9c790 100644 --- a/storm/src/main/storm/mesos/resources/ResourceEntries.java +++ b/storm/src/main/storm/mesos/resources/ResourceEntries.java @@ -93,15 +93,15 @@ public ReservationType getReservationType() { public RangeResourceEntry add(ResourceEntry resourceEntry) { RangeResourceEntry rangeResourceEntry = (RangeResourceEntry) resourceEntry; - if (rangeResourceEntry.getBegin() > this.end || rangeResourceEntry.getEnd() < this.begin) { + if (rangeResourceEntry.getEnd() < rangeResourceEntry.getEnd() || rangeResourceEntry.getBegin() > this.end + 1 || rangeResourceEntry.getEnd() < this.begin - 1) { return this; } - if (this.begin < rangeResourceEntry.getBegin() || this.begin == rangeResourceEntry.getEnd() + 1) { + if (rangeResourceEntry.getBegin() < this.begin) { this.begin = rangeResourceEntry.getBegin(); } - if (this.end > rangeResourceEntry.getBegin() || this.end == rangeResourceEntry.getBegin() + 1) { - this.end = rangeResourceEntry.getBegin(); + if (rangeResourceEntry.getEnd() > this.end) { + this.end = rangeResourceEntry.getEnd(); } return this; } diff --git a/storm/src/test/storm/mesos/resources/ResourceEntryTest.java b/storm/src/test/storm/mesos/resources/ResourceEntryTest.java new file mode 100644 index 000000000..6c24c153c --- /dev/null +++ b/storm/src/test/storm/mesos/resources/ResourceEntryTest.java @@ -0,0 +1,137 @@ +/** + * 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 storm.mesos.resources; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.runners.MockitoJUnitRunner; + +import static org.junit.Assert.assertTrue; +import static storm.mesos.resources.ResourceEntries.RangeResourceEntry; + +@RunWith(MockitoJUnitRunner.class) +public class ResourceEntryTest { + RangeResourceEntry existingRange; + + private boolean isEqual(RangeResourceEntry r1, RangeResourceEntry r2) { + return r1.getBegin().equals(r2.getBegin()) && r1.getEnd().equals(r2.getEnd()); + } + + @Test + public void testDisjointAndNonAdjacentRanges() { + Long beginPort = 1000l; + Long endPort = 2000l; + + existingRange = new RangeResourceEntry(beginPort, endPort); + RangeResourceEntry originalRange = new RangeResourceEntry(beginPort, endPort); + + RangeResourceEntry newRange = new RangeResourceEntry(beginPort - 1000, beginPort - 2); + existingRange.add(newRange); + assertTrue(isEqual(originalRange, existingRange)); + + newRange = new RangeResourceEntry(endPort + 2, endPort + 1000); + existingRange.add(newRange); + assertTrue(isEqual(originalRange, existingRange)); + } + + @Test + public void testDisjointAndAdjacentRanges() { + Long beginPort = 1000l; + Long endPort = 2000l; + + existingRange = new RangeResourceEntry(beginPort, endPort); + + RangeResourceEntry newRange = new RangeResourceEntry(endPort + 1, endPort + 1000); + RangeResourceEntry expectedRange = new RangeResourceEntry(beginPort, endPort + 1000); + existingRange.add(newRange); + assertTrue(isEqual(expectedRange, existingRange)); + + existingRange = new RangeResourceEntry(beginPort, endPort); + newRange = new RangeResourceEntry(beginPort - 1000, beginPort - 1); + expectedRange = new RangeResourceEntry(beginPort - 1000, endPort); + existingRange.add(newRange); + assertTrue(isEqual(expectedRange, existingRange)); + } + + @Test + public void testOverlappingRanges() { + Long beginPort = 1000l; + Long endPort = 2000l; + RangeResourceEntry newRange; + RangeResourceEntry expectedRange; + + existingRange = new RangeResourceEntry(beginPort, endPort); + + existingRange = new RangeResourceEntry(beginPort, endPort); + newRange = new RangeResourceEntry(beginPort - 500, beginPort + 500); + expectedRange = new RangeResourceEntry(beginPort - 500, endPort); + existingRange.add(newRange); + assertTrue(isEqual(expectedRange, existingRange)); + + existingRange = new RangeResourceEntry(beginPort, endPort); + newRange = new RangeResourceEntry(beginPort + 500, endPort + 500); + expectedRange = new RangeResourceEntry(beginPort, endPort + 500); + existingRange.add(newRange); + assertTrue(isEqual(expectedRange, existingRange)); + } + + @Test + public void testSubSets() { + Long beginPort = 1000l; + Long endPort = 2000l; + RangeResourceEntry newRange; + RangeResourceEntry expectedRange; + + existingRange = new RangeResourceEntry(beginPort, endPort); + + newRange = new RangeResourceEntry(beginPort + 250, beginPort + 500); + expectedRange = new RangeResourceEntry(beginPort, endPort); + existingRange.add(newRange); + assertTrue(isEqual(expectedRange, existingRange)); + } + + @Test + public void testSuperSets() { + Long beginPort = 1000l; + Long endPort = 2000l; + RangeResourceEntry newRange; + RangeResourceEntry expectedRange; + + existingRange = new RangeResourceEntry(beginPort, endPort); + + newRange = new RangeResourceEntry(beginPort - 250, endPort + 250); + expectedRange = new RangeResourceEntry(beginPort - 250, endPort + 250); + existingRange.add(newRange); + assertTrue(isEqual(expectedRange, existingRange)); + } + + @Test + public void testInvalidRange() { + Long beginPort = 1000l; + Long endPort = 2000l; + RangeResourceEntry newRange; + RangeResourceEntry expectedRange; + + existingRange = new RangeResourceEntry(beginPort, endPort); + + newRange = new RangeResourceEntry(endPort, beginPort); + expectedRange = new RangeResourceEntry(beginPort, endPort); + existingRange.add(newRange); + assertTrue(isEqual(expectedRange, existingRange)); + } +} From 9ac95805329764d4ca04eaf26543f21df87bed70 Mon Sep 17 00:00:00 2001 From: Karthick Duraisamy Soundararaj Date: Fri, 17 Jun 2016 20:26:42 +0530 Subject: [PATCH 11/12] Renaming OfferResources -> AggregatedOffers --- storm/src/main/storm/mesos/MesosNimbus.java | 38 ++++++------ ...erResources.java => AggregatedOffers.java} | 8 +-- .../mesos/schedulers/DefaultScheduler.java | 22 +++---- .../mesos/schedulers/SchedulerUtils.java | 18 +++--- .../main/storm/mesos/util/MesosCommon.java | 22 +++---- .../src/test/storm/mesos/MesosCommonTest.java | 26 ++++----- .../src/test/storm/mesos/MesosNimbusTest.java | 58 +++++++++---------- .../storm/mesos/OfferRoleComparatorTest.java | 10 ++-- storm/src/test/storm/mesos/TestUtils.java | 18 +++--- ...cesTest.java => AggregatedOffersTest.java} | 30 +++++----- 10 files changed, 125 insertions(+), 125 deletions(-) rename storm/src/main/storm/mesos/resources/{OfferResources.java => AggregatedOffers.java} (97%) rename storm/src/test/storm/mesos/resources/{OfferResourcesTest.java => AggregatedOffersTest.java} (69%) diff --git a/storm/src/main/storm/mesos/MesosNimbus.java b/storm/src/main/storm/mesos/MesosNimbus.java index 5c9a0ad6f..7f52b25fa 100644 --- a/storm/src/main/storm/mesos/MesosNimbus.java +++ b/storm/src/main/storm/mesos/MesosNimbus.java @@ -50,7 +50,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.yaml.snakeyaml.Yaml; -import storm.mesos.resources.OfferResources; +import storm.mesos.resources.AggregatedOffers; import storm.mesos.resources.ReservationType; import storm.mesos.resources.ResourceEntries.RangeResourceEntry; import storm.mesos.resources.ResourceEntries.ScalarResourceEntry; @@ -415,16 +415,16 @@ public void assignSlots(Topologies topologies, Map offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(_offers); - Map> tasksToLaunchPerNode = getTasksToLaunch(topologies, slotsForTopologiesNeedingAssignments, offerResourcesPerNode); + Map aggregatedOffersPerNode = MesosCommon.getAggregatedOffersPerNode(_offers); + Map> tasksToLaunchPerNode = getTasksToLaunch(topologies, slotsForTopologiesNeedingAssignments, aggregatedOffersPerNode); for (String node : tasksToLaunchPerNode.keySet()) { - List offerIDList = offerResourcesPerNode.get(node).getOfferIDList(); + List offerIDList = aggregatedOffersPerNode.get(node).getOfferIDList(); List taskInfoList = tasksToLaunchPerNode.get(node); LOG.info("Using offerIDs: " + offerIDListToString(offerIDList) + " on host: " + node + " to launch tasks: " + taskInfoListToString(taskInfoList)); @@ -557,7 +557,7 @@ private ExecutorInfo.Builder getExecutorInfoBuilder(TopologyDetails details, Str public Map> getTasksToLaunch(Topologies topologies, Map> slots, - Map offerResourcesPerNode) { + Map aggregatedOffersPerNode) { Map> tasksToLaunchPerNode = new HashMap<>(); for (String topologyId : slots.keySet()) { @@ -577,7 +577,7 @@ public Map> getTasksToLaunch(Topologies topologies, String workerHost = slot.getNodeId(); Long workerPort = Long.valueOf(slot.getPort()); - OfferResources offerResources = offerResourcesPerNode.get(slot.getNodeId()); + AggregatedOffers aggregatedOffers = aggregatedOffersPerNode.get(slot.getNodeId()); String workerPrefix = ""; if (mesosStormConf.containsKey(MesosCommon.WORKER_NAME_PREFIX)) { workerPrefix = MesosCommon.getWorkerPrefix(mesosStormConf, topologyDetails); @@ -598,9 +598,9 @@ public Map> getTasksToLaunch(Topologies topologies, String executorDataStr = JSONValue.toJSONString(executorData); String extraConfig = ""; - if (!offerResources.isFit(mesosStormConf, topologyDetails, workerPort, hostsWithSupervisors.contains(workerHost))) { - LOG.error(String.format("Unable to launch worker %s. Required cpu: %f, Required mem: %f. Available OfferResources : %s", - workerHost, requiredCpu, requiredMem, offerResources)); + if (!aggregatedOffers.isFit(mesosStormConf, topologyDetails, workerPort, hostsWithSupervisors.contains(workerHost))) { + LOG.error(String.format("Unable to launch worker %s. Required cpu: %f, Required mem: %f. Available aggregatedOffers : %s", + workerHost, requiredCpu, requiredMem, aggregatedOffers)); continue; } @@ -615,26 +615,26 @@ public Map> getTasksToLaunch(Topologies topologies, executorResources.add(createMesosScalarResource(ResourceType.CPU, new ScalarResourceEntry(executorCpu))); executorResources.add(createMesosScalarResource(ResourceType.MEM, new ScalarResourceEntry(executorMem))); } else { - scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.CPU, new ScalarResourceEntry(executorCpu)); + scalarResourceEntryList = aggregatedOffers.reserveAndGet(ResourceType.CPU, new ScalarResourceEntry(executorCpu)); executorResources.addAll(createMesosScalarResourceList(ResourceType.CPU, scalarResourceEntryList)); - scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.MEM, new ScalarResourceEntry(executorMem)); + scalarResourceEntryList = aggregatedOffers.reserveAndGet(ResourceType.MEM, new ScalarResourceEntry(executorMem)); executorResources.addAll(createMesosScalarResourceList(ResourceType.MEM, scalarResourceEntryList)); } String supervisorStormLocalDir = getStormLocalDirForWorkers(); extraConfig += String.format(" -c storm.local.dir=%s", supervisorStormLocalDir); - scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.CPU, new ScalarResourceEntry(workerCpu)); + scalarResourceEntryList = aggregatedOffers.reserveAndGet(ResourceType.CPU, new ScalarResourceEntry(workerCpu)); workerResources.addAll(createMesosScalarResourceList(ResourceType.CPU, scalarResourceEntryList)); - scalarResourceEntryList = offerResources.reserveAndGet(ResourceType.MEM, new ScalarResourceEntry(workerMem)); + scalarResourceEntryList = aggregatedOffers.reserveAndGet(ResourceType.MEM, new ScalarResourceEntry(workerMem)); workerResources.addAll(createMesosScalarResourceList(ResourceType.MEM, scalarResourceEntryList)); - rangeResourceEntryList = offerResources.reserveAndGet(ResourceType.PORTS, new RangeResourceEntry(workerPort, workerPort)); + rangeResourceEntryList = aggregatedOffers.reserveAndGet(ResourceType.PORTS, new RangeResourceEntry(workerPort, workerPort)); for (ResourceEntry resourceEntry : rangeResourceEntryList) { workerResources.add(createMesosRangeResource(ResourceType.PORTS, (RangeResourceEntry) resourceEntry)); } } catch (ResourceNotAvailableException rexp) { - LOG.warn("Unable to launch worker %s. Required cpu: %f, Required mem: %f. Available OfferResources : %s", - workerHost, requiredCpu, requiredMem, offerResources); + LOG.warn("Unable to launch worker %s. Required cpu: %f, Required mem: %f. Available aggregatedOffers : %s", + workerHost, requiredCpu, requiredMem, aggregatedOffers); continue; } @@ -648,7 +648,7 @@ public Map> getTasksToLaunch(Topologies topologies, TaskInfo task = TaskInfo.newBuilder() .setTaskId(taskId) .setName(taskName) - .setSlaveId(offerResources.getSlaveID()) + .setSlaveId(aggregatedOffers.getSlaveID()) .setExecutor(executorInfoBuilder.build()) .addAllResources(workerResources) .build(); diff --git a/storm/src/main/storm/mesos/resources/OfferResources.java b/storm/src/main/storm/mesos/resources/AggregatedOffers.java similarity index 97% rename from storm/src/main/storm/mesos/resources/OfferResources.java rename to storm/src/main/storm/mesos/resources/AggregatedOffers.java index 4cafff737..bc03339a0 100644 --- a/storm/src/main/storm/mesos/resources/OfferResources.java +++ b/storm/src/main/storm/mesos/resources/AggregatedOffers.java @@ -30,9 +30,9 @@ import java.util.Map; // TODO(ksoundararaj): a better name? - OfferAggregator? -public class OfferResources { +public class AggregatedOffers { - private final Logger log = LoggerFactory.getLogger(OfferResources.class); + private final Logger log = LoggerFactory.getLogger(AggregatedOffers.class); private Map availableResources; private List offerList = new ArrayList(); @@ -48,12 +48,12 @@ private void initializeAvailableResources() { availableResources.put(ResourceType.PORTS, new RangeResource(ResourceType.PORTS)); } - public OfferResources(String hostName) { + public AggregatedOffers(String hostName) { this.hostName = hostName; initializeAvailableResources(); } - public OfferResources(Protos.Offer offer) { + public AggregatedOffers(Protos.Offer offer) { initializeAvailableResources(); this.slaveID = offer.getSlaveId(); this.hostName = offer.getHostname(); diff --git a/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java b/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java index fc1d4d33b..25d08d39b 100644 --- a/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java +++ b/storm/src/main/storm/mesos/schedulers/DefaultScheduler.java @@ -27,7 +27,7 @@ import org.apache.mesos.Protos; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import storm.mesos.resources.OfferResources; +import storm.mesos.resources.AggregatedOffers; import storm.mesos.resources.ResourceNotAvailableException; import storm.mesos.util.MesosCommon; import storm.mesos.util.RotatingMap; @@ -55,7 +55,7 @@ public void prepare(Map conf) { mesosStormConf = conf; } - private List getMesosWorkerSlots(Map offerResourcesPerNode, + private List getMesosWorkerSlots(Map aggregatedOffersPerNode, Collection nodesWithExistingSupervisors, TopologyDetails topologyDetails) { @@ -68,22 +68,22 @@ private List getMesosWorkerSlots(Map of do { slotFound = false; - for (String currentNode : offerResourcesPerNode.keySet()) { - OfferResources offerResources = offerResourcesPerNode.get(currentNode); + for (String currentNode : aggregatedOffersPerNode.keySet()) { + AggregatedOffers aggregatedOffers = aggregatedOffersPerNode.get(currentNode); boolean supervisorExists = nodesWithExistingSupervisors.contains(currentNode); - if (!offerResources.isFit(mesosStormConf, topologyDetails, supervisorExists)) { + if (!aggregatedOffers.isFit(mesosStormConf, topologyDetails, supervisorExists)) { log.info("{} is not a fit for {} requestedWorkerCpu: {} requestedWorkerMem: {}", - offerResources.toString(), topologyDetails.getId(), requestedWorkerCpu, requestedWorkerMem); + aggregatedOffers.toString(), topologyDetails.getId(), requestedWorkerCpu, requestedWorkerMem); continue; } - log.info("{} is a fit for {} requestedWorkerCpu: {} requestedWorkerMem: {}", offerResources.toString(), + log.info("{} is a fit for {} requestedWorkerCpu: {} requestedWorkerMem: {}", aggregatedOffers.toString(), topologyDetails.getId(), requestedWorkerCpu, requestedWorkerMem); MesosWorkerSlot mesosWorkerSlot; try { - mesosWorkerSlot = SchedulerUtils.createMesosWorkerSlot(mesosStormConf, offerResources, topologyDetails, supervisorExists); + mesosWorkerSlot = SchedulerUtils.createMesosWorkerSlot(mesosStormConf, aggregatedOffers, topologyDetails, supervisorExists); } catch (ResourceNotAvailableException rexp) { log.warn(rexp.getMessage()); continue; @@ -130,7 +130,7 @@ public List allSlotsAvailableForScheduling(RotatingMap allSlots = new ArrayList<>(); - Map offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(offers); + Map aggregatedOffersPerNode = MesosCommon.getAggregatedOffersPerNode(offers); for (String currentTopology : topologiesMissingAssignments) { TopologyDetails topologyDetails = topologies.getById(currentTopology); @@ -142,13 +142,13 @@ public List allSlotsAvailableForScheduling(RotatingMap nodesWithExistingSupervisors = new HashSet<>(); - for (String currentNode : offerResourcesPerNode.keySet()) { + for (String currentNode : aggregatedOffersPerNode.keySet()) { if (SchedulerUtils.supervisorExists(currentNode, existingSupervisors, currentTopology)) { nodesWithExistingSupervisors.add(currentNode); } } - List mesosWorkerSlotList = getMesosWorkerSlots(offerResourcesPerNode, nodesWithExistingSupervisors, topologyDetails); + List mesosWorkerSlotList = getMesosWorkerSlots(aggregatedOffersPerNode, nodesWithExistingSupervisors, topologyDetails); for (MesosWorkerSlot mesosWorkerSlot : mesosWorkerSlotList) { String slotId = String.format("%s:%s", mesosWorkerSlot.getNodeId(), mesosWorkerSlot.getPort()); mesosWorkerSlotMap.put(slotId, mesosWorkerSlot); diff --git a/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java b/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java index 4c61355a4..4b1c36479 100644 --- a/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java +++ b/storm/src/main/storm/mesos/schedulers/SchedulerUtils.java @@ -19,7 +19,7 @@ import backtype.storm.scheduler.SupervisorDetails; import backtype.storm.scheduler.TopologyDetails; -import storm.mesos.resources.OfferResources; +import storm.mesos.resources.AggregatedOffers; import storm.mesos.resources.ResourceNotAvailableException; import storm.mesos.resources.ResourceType; import storm.mesos.util.MesosCommon; @@ -34,9 +34,9 @@ public class SchedulerUtils { - public static List getPorts(OfferResources offerResources, int requiredCount) { + public static List getPorts(AggregatedOffers aggregatedOffers, int requiredCount) { List retVal = new ArrayList<>(); - List resourceEntryList = offerResources.getAllAvailableResources(ResourceType.PORTS); + List resourceEntryList = aggregatedOffers.getAllAvailableResources(ResourceType.PORTS); for (RangeResourceEntry rangeResourceEntry : resourceEntryList) { Long begin = rangeResourceEntry.getBegin(); @@ -51,7 +51,7 @@ public static List getPorts(OfferResources offerResources, i } public static MesosWorkerSlot createMesosWorkerSlot(Map mesosStormConf, - OfferResources offerResources, + AggregatedOffers aggregatedOffers, TopologyDetails topologyDetails, boolean supervisorExists) throws ResourceNotAvailableException { @@ -61,16 +61,16 @@ public static MesosWorkerSlot createMesosWorkerSlot(Map mesosStormConf, requestedWorkerCpu += supervisorExists ? 0 : MesosCommon.executorCpu(mesosStormConf); requestedWorkerMem += supervisorExists ? 0 : MesosCommon.executorMem(mesosStormConf); - offerResources.reserve(ResourceType.CPU, new ScalarResourceEntry(requestedWorkerCpu)); - offerResources.reserve(ResourceType.MEM, new ScalarResourceEntry(requestedWorkerMem)); + aggregatedOffers.reserve(ResourceType.CPU, new ScalarResourceEntry(requestedWorkerCpu)); + aggregatedOffers.reserve(ResourceType.MEM, new ScalarResourceEntry(requestedWorkerMem)); - List ports = getPorts(offerResources, 1); + List ports = getPorts(aggregatedOffers, 1); if (ports.isEmpty()) { throw new ResourceNotAvailableException("No ports available to create MesosWorkerSlot."); } - offerResources.reserve(ResourceType.PORTS, ports.get(0)); + aggregatedOffers.reserve(ResourceType.PORTS, ports.get(0)); - return new MesosWorkerSlot(offerResources.getHostName(), ports.get(0).getBegin(), topologyDetails.getId()); + return new MesosWorkerSlot(aggregatedOffers.getHostName(), ports.get(0).getBegin(), topologyDetails.getId()); } /** diff --git a/storm/src/main/storm/mesos/util/MesosCommon.java b/storm/src/main/storm/mesos/util/MesosCommon.java index efa853f37..ae9946ee8 100644 --- a/storm/src/main/storm/mesos/util/MesosCommon.java +++ b/storm/src/main/storm/mesos/util/MesosCommon.java @@ -23,7 +23,7 @@ import org.apache.mesos.Protos; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import storm.mesos.resources.OfferResources; +import storm.mesos.resources.AggregatedOffers; import java.net.InetAddress; import java.net.UnknownHostException; @@ -138,25 +138,25 @@ public static int portFromTaskId(String taskId) { return port; } - public static Map getConsolidatedOfferResourcesPerNode(RotatingMap offers) { - Map offerResourcesPerNode = new HashMap<>(); + public static Map getAggregatedOffersPerNode(RotatingMap offers) { + Map aggregatedOffersPerNode = new HashMap<>(); for (Protos.Offer offer : offers.values()) { String hostName = offer.getHostname(); - OfferResources offerResources = offerResourcesPerNode.get(hostName); - if (offerResources == null) { - offerResources = new OfferResources(offer); - offerResourcesPerNode.put(hostName, offerResources); + AggregatedOffers aggregatedOffers = aggregatedOffersPerNode.get(hostName); + if (aggregatedOffers == null) { + aggregatedOffers = new AggregatedOffers(offer); + aggregatedOffersPerNode.put(hostName, aggregatedOffers); } else { - offerResources.add(offer); + aggregatedOffers.add(offer); } } - for (OfferResources offerResources : offerResourcesPerNode.values()) { - LOG.info("Available resources at {}: {}", offerResources.getHostName(), offerResources.toString()); + for (AggregatedOffers aggregatedOffers : aggregatedOffersPerNode.values()) { + LOG.info("Available resources at {}: {}", aggregatedOffers.getHostName(), aggregatedOffers.toString()); } - return offerResourcesPerNode; + return aggregatedOffersPerNode; } public static int getSuicideTimeout(Map conf) { diff --git a/storm/src/test/storm/mesos/MesosCommonTest.java b/storm/src/test/storm/mesos/MesosCommonTest.java index d177a483f..d5c3f6cfb 100644 --- a/storm/src/test/storm/mesos/MesosCommonTest.java +++ b/storm/src/test/storm/mesos/MesosCommonTest.java @@ -22,7 +22,7 @@ import org.apache.mesos.Protos; import org.junit.Before; import org.junit.Test; -import storm.mesos.resources.OfferResources; +import storm.mesos.resources.AggregatedOffers; import storm.mesos.resources.ReservationType; import storm.mesos.resources.ResourceType; import storm.mesos.util.MesosCommon; @@ -330,7 +330,7 @@ public void testExecutorMem() throws Exception { } @Test - public void getConsolidatedOfferResourcesPerNode() { + public void aggregatedOffersPerNode() { RotatingMap r = new RotatingMap(2); Protos.Offer offer = TestUtils.buildOffer("0-1", "h1", 0, 0); r.put(offer.getId(), offer); @@ -344,16 +344,16 @@ public void getConsolidatedOfferResourcesPerNode() { offer = TestUtils.buildOfferWithPorts("O-4", "h2", 0, 0, 1, 100); r.put(offer.getId(), offer); - Map offerResourcesMap = MesosCommon.getConsolidatedOfferResourcesPerNode(r); - OfferResources offerResources = offerResourcesMap.get("h1"); - assertEquals(210, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); - assertEquals(3000, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); - assertEquals(200, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU, ReservationType.STATICALLY_RESERVED), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); - assertEquals(2000, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM, ReservationType.STATICALLY_RESERVED), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); - assertEquals(0, TestUtils.calculateAllAvailableRangeResources(offerResources, ResourceType.PORTS, ReservationType.STATICALLY_RESERVED).size(), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); - offerResources = offerResourcesMap.get("h2"); - assertEquals(0, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU, ReservationType.STATICALLY_RESERVED), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); - assertEquals(0, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM, ReservationType.STATICALLY_RESERVED), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); - assertEquals(100, TestUtils.calculateAllAvailableRangeResources(offerResources, ResourceType.PORTS, ReservationType.UNRESERVED).size(), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); + Map aggregatedOffersPerNode = MesosCommon.getAggregatedOffersPerNode(r); + AggregatedOffers aggregatedOffers = aggregatedOffersPerNode.get("h1"); + assertEquals(210, TestUtils.calculateAllAvailableScalarResources(aggregatedOffers, ResourceType.CPU), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(3000, TestUtils.calculateAllAvailableScalarResources(aggregatedOffers, ResourceType.MEM), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(200, TestUtils.calculateAllAvailableScalarResources(aggregatedOffers, ResourceType.CPU, ReservationType.STATICALLY_RESERVED), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(2000, TestUtils.calculateAllAvailableScalarResources(aggregatedOffers, ResourceType.MEM, ReservationType.STATICALLY_RESERVED), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(0, TestUtils.calculateAllAvailableRangeResources(aggregatedOffers, ResourceType.PORTS, ReservationType.STATICALLY_RESERVED).size(), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); + aggregatedOffers = aggregatedOffersPerNode.get("h2"); + assertEquals(0, TestUtils.calculateAllAvailableScalarResources(aggregatedOffers, ResourceType.CPU, ReservationType.STATICALLY_RESERVED), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(0, TestUtils.calculateAllAvailableScalarResources(aggregatedOffers, ResourceType.MEM, ReservationType.STATICALLY_RESERVED), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(100, TestUtils.calculateAllAvailableRangeResources(aggregatedOffers, ResourceType.PORTS, ReservationType.UNRESERVED).size(), MesosCommonTest.DELTA_FOR_DOUBLE_COMPARISON); } } diff --git a/storm/src/test/storm/mesos/MesosNimbusTest.java b/storm/src/test/storm/mesos/MesosNimbusTest.java index 2c3682f08..b4327b886 100644 --- a/storm/src/test/storm/mesos/MesosNimbusTest.java +++ b/storm/src/test/storm/mesos/MesosNimbusTest.java @@ -26,7 +26,7 @@ import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; -import storm.mesos.resources.OfferResources; +import storm.mesos.resources.AggregatedOffers; import storm.mesos.resources.ResourceType; import storm.mesos.util.MesosCommon; import storm.mesos.util.RotatingMap; @@ -222,10 +222,10 @@ public void testGetTasksToLaunchWhenNoTopologiesNeedAssignments() { Topologies topologies = new Topologies(topologyDetailsMap); - Map offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + Map aggregatedOffersPerNode = MesosCommon.getAggregatedOffersPerNode(rotatingMap); Map> workerSlotsMap = new HashMap<>(); - Map> tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + Map> tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, aggregatedOffersPerNode); assertTrue(tasksToLaunch.isEmpty()); } @@ -246,13 +246,13 @@ public void testGetTasksToLaunchForOneTopologyWithOneOffer() { Offer offer = TestUtils.buildOfferWithPorts("O-1", "h1", 24, 40000, 3100, 3200); rotatingMap.put(offer.getId(), offer); - Map offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + Map aggregatedOffersPerNode = MesosCommon.getAggregatedOffersPerNode(rotatingMap); Map> workerSlotsMap = new HashMap<>(); Collection workerSlots = new ArrayList<>(); workerSlots.add(new WorkerSlot("h1", 3100)); workerSlotsMap.put("t1", workerSlots); - Map> tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + Map> tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, aggregatedOffersPerNode); assertTrue((tasksToLaunch.size() == 1)); assertTrue((tasksToLaunch.get("h1").size() == 1)); @@ -261,75 +261,75 @@ public void testGetTasksToLaunchForOneTopologyWithOneOffer() { offer = TestUtils.buildOfferWithReservationAndPorts("O-1", "h1", 0.75, 750, 0.75, 850, 3100, 3101); rotatingMap.put(offer.getId(), offer); - offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + aggregatedOffersPerNode = MesosCommon.getAggregatedOffersPerNode(rotatingMap); workerSlotsMap = new HashMap<>(); workerSlots = new ArrayList<>(); workerSlots.add(new WorkerSlot("h1", 3100)); workerSlotsMap.put("t1", workerSlots); - tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, aggregatedOffersPerNode); assertTrue((tasksToLaunch.size() == 1)); assertTrue((tasksToLaunch.get("h1").size() == 1)); assertTrue(hasResources(FRAMEWORK_ROLE, tasksToLaunch.get("h1").get(0), 0.75 - MesosCommon.DEFAULT_EXECUTOR_CPU, 850 - MesosCommon.DEFAULT_EXECUTOR_MEM_MB)); assertTrue(hasResources("*", tasksToLaunch.get("h1").get(0), 0.35, 650.0)); assertTrue(hasCorrectExecutorResources(tasksToLaunch.get("h1"))); - assertEquals(0.4f, TestUtils.calculateAllAvailableScalarResources(offerResourcesPerNode.get("h1"), ResourceType.CPU), 0.01f); - assertEquals(100f, TestUtils.calculateAllAvailableScalarResources(offerResourcesPerNode.get("h1"), ResourceType.MEM), 0.01f); + assertEquals(0.4f, TestUtils.calculateAllAvailableScalarResources(aggregatedOffersPerNode.get("h1"), ResourceType.CPU), 0.01f); + assertEquals(100f, TestUtils.calculateAllAvailableScalarResources(aggregatedOffersPerNode.get("h1"), ResourceType.MEM), 0.01f); // One offer with only reserved resources offer = TestUtils.buildOfferWithReservationAndPorts("O-1", "h1", 0, 0, 1.5, 1600, 3100, 3101); rotatingMap.put(offer.getId(), offer); - offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + aggregatedOffersPerNode = MesosCommon.getAggregatedOffersPerNode(rotatingMap); workerSlotsMap = new HashMap<>(); workerSlots = new ArrayList<>(); workerSlots.add(new WorkerSlot("h1", 3100)); workerSlotsMap.put("t1", workerSlots); - tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, aggregatedOffersPerNode); assertTrue((tasksToLaunch.size() == 1)); assertTrue(tasksToLaunch.get("h1").size() == 1); assertTrue(hasResources(FRAMEWORK_ROLE, tasksToLaunch.get("h1").get(0), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB)); assertTrue(hasCorrectExecutorResources(tasksToLaunch.get("h1"))); - assertEquals(TestUtils.calculateAllAvailableScalarResources(offerResourcesPerNode.get("h1"), ResourceType.CPU), 0.4f, 0.01f); - assertEquals(TestUtils.calculateAllAvailableScalarResources(offerResourcesPerNode.get("h1"), ResourceType.MEM), 100f, 0.01f); + assertEquals(TestUtils.calculateAllAvailableScalarResources(aggregatedOffersPerNode.get("h1"), ResourceType.CPU), 0.4f, 0.01f); + assertEquals(TestUtils.calculateAllAvailableScalarResources(aggregatedOffersPerNode.get("h1"), ResourceType.MEM), 100f, 0.01f); // Offer with Insufficient cpu offer = TestUtils.buildOfferWithPorts("O-1", "h1", 0, 40000, 3100, 3200); rotatingMap.put(offer.getId(), offer); - offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + aggregatedOffersPerNode = MesosCommon.getAggregatedOffersPerNode(rotatingMap); - tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, aggregatedOffersPerNode); assertTrue(tasksToLaunch.isEmpty()); // Offer with Insufficient Mem for both executor and worker combined offer = TestUtils.buildOfferWithPorts("O-1", "h1", 24, 900, 3100, 3200); rotatingMap.put(offer.getId(), offer); - offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + aggregatedOffersPerNode = MesosCommon.getAggregatedOffersPerNode(rotatingMap); - tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, aggregatedOffersPerNode); assertTrue(tasksToLaunch.isEmpty()); // Offer with Insufficient Mem for executor offer = TestUtils.buildOfferWithPorts("O-1", "h1", 24, 1400, 3100, 3200); rotatingMap.put(offer.getId(), offer); - offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + aggregatedOffersPerNode = MesosCommon.getAggregatedOffersPerNode(rotatingMap); - tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, aggregatedOffersPerNode); assertTrue(tasksToLaunch.isEmpty()); // One offer with Insufficient ports offer = TestUtils.buildOffer("O-1", "h1", 24, 4000); rotatingMap.put(offer.getId(), offer); - offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + aggregatedOffersPerNode = MesosCommon.getAggregatedOffersPerNode(rotatingMap); - tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, aggregatedOffersPerNode); assertTrue(tasksToLaunch.isEmpty()); } @@ -351,22 +351,22 @@ public void testGetTasksToLaunchForOneTopologyWithMultipleOffersOnSameHost() { offer = TestUtils.buildOfferWithPorts("O-3", "h1", 0, 0, 3100, 3200); rotatingMap.put(offer.getId(), offer); - Map offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + Map aggregatedOffersPerNode = MesosCommon.getAggregatedOffersPerNode(rotatingMap); Map> workerSlotsMap = new HashMap<>(); Collection workerSlots = new ArrayList<>(); workerSlots.add(new WorkerSlot("h1", 3100)); workerSlotsMap.put("t1", workerSlots); - Map> tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + Map> tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, aggregatedOffersPerNode); assertTrue((tasksToLaunch.size() == 1)); assertTrue((tasksToLaunch.get("h1").size() == 1)); List taskInfoList = tasksToLaunch.get("h1"); assertTrue(hasResources("*", taskInfoList.get(0), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3100l)); assertTrue(hasCorrectExecutorResources(taskInfoList)); - offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + aggregatedOffersPerNode = MesosCommon.getAggregatedOffersPerNode(rotatingMap); workerSlots.add(new WorkerSlot("h1", 3101)); workerSlots.add(new WorkerSlot("h1", 3102)); - tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, aggregatedOffersPerNode); taskInfoList = tasksToLaunch.get("h1"); assertTrue(taskInfoList.size() == 3); assertTrue(hasResources("*", taskInfoList.get(0), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3100l)); @@ -388,9 +388,9 @@ public void testGetTasksToLaunchForOneTopologyWithMultipleOffersOnSameHost() { topologyDetailsMap.put("t2", t2); topologies = new Topologies(topologyDetailsMap); - offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + aggregatedOffersPerNode = MesosCommon.getAggregatedOffersPerNode(rotatingMap); - tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, aggregatedOffersPerNode); Map> topologyIDtoTaskInfoMap = getTopologyIDtoTaskInfoMap(tasksToLaunch.get("h1")); taskInfoList = topologyIDtoTaskInfoMap.get("t1"); assertTrue(hasResources("*", taskInfoList.get(0), MesosCommon.DEFAULT_WORKER_CPU, MesosCommon.DEFAULT_WORKER_MEM_MB, 3100l)); @@ -435,7 +435,7 @@ public void testGetTasksToLaunchForOneTopologyWithMultipleOffersAcrossMultipleHo offer = TestUtils.buildOfferWithPorts("O-H2-3", "h2", 0, 0, 3100, 3102); rotatingMap.put(offer.getId(), offer); - Map offerResourcesPerNode = MesosCommon.getConsolidatedOfferResourcesPerNode(rotatingMap); + Map aggregatedOffersPerNode = MesosCommon.getAggregatedOffersPerNode(rotatingMap); Map> workerSlotsMap = new HashMap<>(); Map> tasksToLaunch = new HashMap<>(); @@ -451,7 +451,7 @@ public void testGetTasksToLaunchForOneTopologyWithMultipleOffersAcrossMultipleHo workerSlots.add(new WorkerSlot("h2", 3102)); workerSlotsMap.put("t2", workerSlots); - tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, offerResourcesPerNode); + tasksToLaunch = mesosNimbus.getTasksToLaunch(topologies, workerSlotsMap, aggregatedOffersPerNode); List taskInfoList = new ArrayList<>(); for(List til : tasksToLaunch.values()) { diff --git a/storm/src/test/storm/mesos/OfferRoleComparatorTest.java b/storm/src/test/storm/mesos/OfferRoleComparatorTest.java index 27cf29920..6429997aa 100644 --- a/storm/src/test/storm/mesos/OfferRoleComparatorTest.java +++ b/storm/src/test/storm/mesos/OfferRoleComparatorTest.java @@ -33,18 +33,18 @@ public class OfferRoleComparatorTest { @Test public void testCompare() throws Exception { - List offerResources = new ArrayList<>(); + List aggregatedOffers = new ArrayList<>(); Offer offer = TestUtils.buildOffer(); - offerResources.addAll(offer.getResourcesList()); - Collections.sort(offerResources, new ResourceRoleComparator()); + aggregatedOffers.addAll(offer.getResourcesList()); + Collections.sort(aggregatedOffers, new ResourceRoleComparator()); assertEquals( "*", - offerResources.get(5).getRole() + aggregatedOffers.get(5).getRole() ); assertEquals( "*", - offerResources.get(4).getRole() + aggregatedOffers.get(4).getRole() ); } } diff --git a/storm/src/test/storm/mesos/TestUtils.java b/storm/src/test/storm/mesos/TestUtils.java index c745ca83c..af41974f1 100644 --- a/storm/src/test/storm/mesos/TestUtils.java +++ b/storm/src/test/storm/mesos/TestUtils.java @@ -20,7 +20,7 @@ import backtype.storm.generated.StormTopology; import backtype.storm.scheduler.TopologyDetails; import org.apache.mesos.Protos; -import storm.mesos.resources.OfferResources; +import storm.mesos.resources.AggregatedOffers; import storm.mesos.resources.ReservationType; import storm.mesos.resources.ResourceEntries; import storm.mesos.resources.ResourceEntry; @@ -310,20 +310,20 @@ public static List calculateAllAvailableRangeResources(List return retVal; } - public static double calculateAllAvailableScalarResources(OfferResources offerResources, ResourceType resourceType) { - return calculateAllAvailableScalarResources(offerResources.getAllAvailableResources(resourceType)); + public static double calculateAllAvailableScalarResources(AggregatedOffers aggregatedOffers, ResourceType resourceType) { + return calculateAllAvailableScalarResources(aggregatedOffers.getAllAvailableResources(resourceType)); } - public static List calculateAllAvailableRangeResources(OfferResources offerResources, ResourceType resourceType) { - return calculateAllAvailableRangeResources(offerResources.getAllAvailableResources(resourceType)); + public static List calculateAllAvailableRangeResources(AggregatedOffers aggregatedOffers, ResourceType resourceType) { + return calculateAllAvailableRangeResources(aggregatedOffers.getAllAvailableResources(resourceType)); } - public static double calculateAllAvailableScalarResources(OfferResources offerResources, ResourceType resourceType, ReservationType reservationType) { - return calculateAllAvailableScalarResources(offerResources.getAllAvailableResources(resourceType, reservationType)); + public static double calculateAllAvailableScalarResources(AggregatedOffers aggregatedOffers, ResourceType resourceType, ReservationType reservationType) { + return calculateAllAvailableScalarResources(aggregatedOffers.getAllAvailableResources(resourceType, reservationType)); } - public static List calculateAllAvailableRangeResources(OfferResources offerResources, ResourceType resourceType, ReservationType reservationType) { - return calculateAllAvailableRangeResources(offerResources.getAllAvailableResources(resourceType, reservationType)); + public static List calculateAllAvailableRangeResources(AggregatedOffers aggregatedOffers, ResourceType resourceType, ReservationType reservationType) { + return calculateAllAvailableRangeResources(aggregatedOffers.getAllAvailableResources(resourceType, reservationType)); } } diff --git a/storm/src/test/storm/mesos/resources/OfferResourcesTest.java b/storm/src/test/storm/mesos/resources/AggregatedOffersTest.java similarity index 69% rename from storm/src/test/storm/mesos/resources/OfferResourcesTest.java rename to storm/src/test/storm/mesos/resources/AggregatedOffersTest.java index ed873054d..bad9f972c 100644 --- a/storm/src/test/storm/mesos/resources/OfferResourcesTest.java +++ b/storm/src/test/storm/mesos/resources/AggregatedOffersTest.java @@ -29,7 +29,7 @@ import static org.junit.Assert.assertTrue; @RunWith(MockitoJUnitRunner.class) -public class OfferResourcesTest { +public class AggregatedOffersTest { private static final double DELTA_FOR_DOUBLE_COMPARISON = 0.0001; @Test @@ -40,26 +40,26 @@ public void testToIgnoreDynamicResources() { // Note that buidOffer adds Offer offer = TestUtils.buildOffer("0-1", "h1", 0, 0); - OfferResources offerResources = new OfferResources(offer); + AggregatedOffers aggregatedOffers = new AggregatedOffers(offer); - assertEquals(0, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU), DELTA_FOR_DOUBLE_COMPARISON); - assertEquals(0, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM), DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(0, TestUtils.calculateAllAvailableScalarResources(aggregatedOffers, ResourceType.CPU), DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(0, TestUtils.calculateAllAvailableScalarResources(aggregatedOffers, ResourceType.MEM), DELTA_FOR_DOUBLE_COMPARISON); - assertTrue(offerResources.getHostName().equals(offer.getHostname())); - assertTrue(offerResources.getSlaveID().equals(offer.getSlaveId())); + assertTrue(aggregatedOffers.getHostName().equals(offer.getHostname())); + assertTrue(aggregatedOffers.getSlaveID().equals(offer.getSlaveId())); offer = TestUtils.buildOfferWithReservation("offer1", "h1", 2, 1000, 6, 1000); - offerResources = new OfferResources(offer); - assertEquals(8, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU), DELTA_FOR_DOUBLE_COMPARISON); - assertEquals(2000, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM), DELTA_FOR_DOUBLE_COMPARISON); - assertTrue(offerResources.getHostName().equals(offer.getHostname())); - assertTrue(offerResources.getSlaveID().equals(offer.getSlaveId())); + aggregatedOffers = new AggregatedOffers(offer); + assertEquals(8, TestUtils.calculateAllAvailableScalarResources(aggregatedOffers, ResourceType.CPU), DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(2000, TestUtils.calculateAllAvailableScalarResources(aggregatedOffers, ResourceType.MEM), DELTA_FOR_DOUBLE_COMPARISON); + assertTrue(aggregatedOffers.getHostName().equals(offer.getHostname())); + assertTrue(aggregatedOffers.getSlaveID().equals(offer.getSlaveId())); offer = TestUtils.buildOfferWithPorts("offer1", "h1", 2.0, 2000, 3000, 3100); - offerResources = new OfferResources(offer); - assertEquals(2.0, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.CPU), DELTA_FOR_DOUBLE_COMPARISON); - assertEquals(2000, TestUtils.calculateAllAvailableScalarResources(offerResources, ResourceType.MEM),DELTA_FOR_DOUBLE_COMPARISON); - List rangeResources = TestUtils.calculateAllAvailableRangeResources(offerResources, ResourceType.PORTS); + aggregatedOffers = new AggregatedOffers(offer); + assertEquals(2.0, TestUtils.calculateAllAvailableScalarResources(aggregatedOffers, ResourceType.CPU), DELTA_FOR_DOUBLE_COMPARISON); + assertEquals(2000, TestUtils.calculateAllAvailableScalarResources(aggregatedOffers, ResourceType.MEM),DELTA_FOR_DOUBLE_COMPARISON); + List rangeResources = TestUtils.calculateAllAvailableRangeResources(aggregatedOffers, ResourceType.PORTS); assertTrue(rangeResources.size() == 101); } } From d3c2a38b26889d02d47c9dab552f527d16f1bd9b Mon Sep 17 00:00:00 2001 From: Karthick Duraisamy Soundararaj Date: Sat, 18 Jun 2016 12:44:35 +0530 Subject: [PATCH 12/12] Addressing Erik's comment regarding 'avaliable' typo and removing TODO --- .../storm/mesos/resources/AggregatedOffers.java | 15 +++++++-------- .../storm/mesos/resources/ResourceEntries.java | 3 ++- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/storm/src/main/storm/mesos/resources/AggregatedOffers.java b/storm/src/main/storm/mesos/resources/AggregatedOffers.java index bc03339a0..95fe0fced 100644 --- a/storm/src/main/storm/mesos/resources/AggregatedOffers.java +++ b/storm/src/main/storm/mesos/resources/AggregatedOffers.java @@ -29,7 +29,6 @@ import java.util.List; import java.util.Map; -// TODO(ksoundararaj): a better name? - OfferAggregator? public class AggregatedOffers { private final Logger log = LoggerFactory.getLogger(AggregatedOffers.class); @@ -98,11 +97,11 @@ public void add(Protos.Offer offer) { } } - public boolean isAvaliable(ResourceType resourceType, ResourceEntry resource) { + public boolean isAvailable(ResourceType resourceType, ResourceEntry resource) { return availableResources.get(resourceType).isAvailable(resource); } - public boolean isAvaliable(ResourceType resourceType, ReservationType reservationType, ResourceEntry resource) { + public boolean isAvailable(ResourceType resourceType, ReservationType reservationType, ResourceEntry resource) { return availableResources.get(resourceType).isAvailable(resource, reservationType); } @@ -166,8 +165,8 @@ public boolean isFit(Map mesosStormConf, TopologyDetails topologyDetails, boolea requestedWorkerCpu += supervisorExists ? 0 : MesosCommon.executorCpu(mesosStormConf); requestedWorkerMem += supervisorExists ? 0 : MesosCommon.executorMem(mesosStormConf); - return (isAvaliable(ResourceType.CPU, new ResourceEntries.ScalarResourceEntry(requestedWorkerCpu)) && - isAvaliable(ResourceType.MEM, new ResourceEntries.ScalarResourceEntry(requestedWorkerMem)) && + return (isAvailable(ResourceType.CPU, new ResourceEntries.ScalarResourceEntry(requestedWorkerCpu)) && + isAvailable(ResourceType.MEM, new ResourceEntries.ScalarResourceEntry(requestedWorkerMem)) && !getAllAvailableResources(ResourceType.PORTS).isEmpty()); } @@ -179,9 +178,9 @@ public boolean isFit(Map mesosStormConf, TopologyDetails topologyDetails, Long p requestedWorkerCpu += supervisorExists ? 0 : MesosCommon.executorCpu(mesosStormConf); requestedWorkerMem += supervisorExists ? 0 : MesosCommon.executorMem(mesosStormConf); - return (isAvaliable(ResourceType.CPU, new ResourceEntries.ScalarResourceEntry(requestedWorkerCpu)) && - isAvaliable(ResourceType.MEM, new ResourceEntries.ScalarResourceEntry(requestedWorkerMem)) && - isAvaliable(ResourceType.PORTS, new ResourceEntries.RangeResourceEntry(port, port))); + return (isAvailable(ResourceType.CPU, new ResourceEntries.ScalarResourceEntry(requestedWorkerCpu)) && + isAvailable(ResourceType.MEM, new ResourceEntries.ScalarResourceEntry(requestedWorkerMem)) && + isAvailable(ResourceType.PORTS, new ResourceEntries.RangeResourceEntry(port, port))); } } diff --git a/storm/src/main/storm/mesos/resources/ResourceEntries.java b/storm/src/main/storm/mesos/resources/ResourceEntries.java index c66e9c790..f71473451 100644 --- a/storm/src/main/storm/mesos/resources/ResourceEntries.java +++ b/storm/src/main/storm/mesos/resources/ResourceEntries.java @@ -93,7 +93,8 @@ public ReservationType getReservationType() { public RangeResourceEntry add(ResourceEntry resourceEntry) { RangeResourceEntry rangeResourceEntry = (RangeResourceEntry) resourceEntry; - if (rangeResourceEntry.getEnd() < rangeResourceEntry.getEnd() || rangeResourceEntry.getBegin() > this.end + 1 || rangeResourceEntry.getEnd() < this.begin - 1) { + if (rangeResourceEntry.getEnd() < rangeResourceEntry.getEnd() || + rangeResourceEntry.getBegin() > this.end + 1 || rangeResourceEntry.getEnd() < this.begin - 1) { return this; }