You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by abellina <gi...@git.apache.org> on 2016/12/16 16:36:46 UTC

[GitHub] storm pull request #1674: STORM-2083: Blacklist scheduler

Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1674#discussion_r92839936
  
    --- Diff: storm-core/src/jvm/org/apache/storm/scheduler/blacklist/BlacklistScheduler.java ---
    @@ -0,0 +1,252 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.scheduler.blacklist;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.metric.StormMetricsRegistry;
    +import org.apache.storm.scheduler.Cluster;
    +import org.apache.storm.scheduler.IScheduler;
    +import org.apache.storm.scheduler.SupervisorDetails;
    +import org.apache.storm.scheduler.Topologies;
    +import org.apache.storm.scheduler.WorkerSlot;
    +import org.apache.storm.scheduler.blacklist.reporters.IReporter;
    +import org.apache.storm.scheduler.blacklist.strategies.IBlacklistStrategy;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.Callable;
    +
    +public class BlacklistScheduler implements IScheduler {
    +    private static final Logger LOG = LoggerFactory.getLogger(BlacklistScheduler.class);
    +    IScheduler underlyingScheduler;
    +    @SuppressWarnings("rawtypes")
    +    private Map _conf;
    +
    +    private int toleranceTime;
    +    private int toleranceCount;
    +    private int resumeTime;
    +    private IReporter reporter;
    +    private IBlacklistStrategy blacklistStrategy;
    +
    +    private int nimbusMonitorFreqSecs;
    +
    +    private Map<String, Set<Integer>> cachedSupervisors;
    +
    +    //key is supervisor key ,value is supervisor ports
    +    private CircularBuffer<HashMap<String, Set<Integer>>> badSupervisorsTolerance;
    +    private Set<String> blacklistHost;
    +
    +    public BlacklistScheduler(IScheduler underlyingScheduler) {
    +        this.underlyingScheduler = underlyingScheduler;
    +    }
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        LOG.info("prepare black list scheduler");
    +        LOG.info(conf.toString());
    +        underlyingScheduler.prepare(conf);
    +        _conf = conf;
    +        if (_conf.containsKey(Config.BLACKLIST_SCHEDULER_TOLERANCE_TIME)) {
    +            toleranceTime = (Integer) _conf.get(Config.BLACKLIST_SCHEDULER_TOLERANCE_TIME);
    +        }
    +        if (_conf.containsKey(Config.BLACKLIST_SCHEDULER_TOLERANCE_COUNT)) {
    +            toleranceCount = (Integer) _conf.get(Config.BLACKLIST_SCHEDULER_TOLERANCE_COUNT);
    +        }
    +        if (_conf.containsKey(Config.BLACKLIST_SCHEDULER_RESUME_TIME)) {
    +            resumeTime = (Integer) _conf.get(Config.BLACKLIST_SCHEDULER_RESUME_TIME);
    +        }
    +        String reporterClassName = _conf.containsKey(Config.BLACKLIST_SCHEDULER_REPORTER) ? (String) _conf.get(Config.BLACKLIST_SCHEDULER_REPORTER) : "";
    +        try {
    +            reporter = (IReporter) Class.forName(reporterClassName).newInstance();
    +        } catch (ClassNotFoundException e) {
    +            LOG.error("Can't find blacklist reporter for name {}", reporterClassName);
    +            throw new RuntimeException(e);
    +        } catch (InstantiationException e) {
    +            LOG.error("Throw InstantiationException blacklist reporter for name {}", reporterClassName);
    +            throw new RuntimeException(e);
    +        } catch (IllegalAccessException e) {
    +            LOG.error("Throw illegalAccessException blacklist reporter for name {}", reporterClassName);
    +            throw new RuntimeException(e);
    +        }
    +
    +        String strategyClassName = _conf.containsKey(Config.BLACKLIST_SCHEDULER_STRATEGY) ? (String) _conf.get(Config.BLACKLIST_SCHEDULER_STRATEGY) : "";
    +        try {
    +            blacklistStrategy = (IBlacklistStrategy) Class.forName(strategyClassName).newInstance();
    +        } catch (ClassNotFoundException e) {
    +            LOG.error("Can't find blacklist strategy for name {}", strategyClassName);
    +            throw new RuntimeException(e);
    +        } catch (InstantiationException e) {
    +            LOG.error("Throw InstantiationException blacklist strategy for name {}", strategyClassName);
    +            throw new RuntimeException(e);
    +        } catch (IllegalAccessException e) {
    +            LOG.error("Throw illegalAccessException blacklist strategy for name {}", strategyClassName);
    +            throw new RuntimeException(e);
    +        }
    +
    +        nimbusMonitorFreqSecs = (Integer) _conf.get(Config.NIMBUS_MONITOR_FREQ_SECS);
    +        blacklistStrategy.prepare(reporter, toleranceTime, toleranceCount, resumeTime, nimbusMonitorFreqSecs);
    +
    +        badSupervisorsTolerance = new CircularBuffer<HashMap<String, Set<Integer>>>(toleranceTime / nimbusMonitorFreqSecs);
    +        cachedSupervisors = new HashMap<>();
    +        blacklistHost = new HashSet<>();
    +
    +        StormMetricsRegistry.registerGauge("nimbus:num-blacklisted-supervisor", new Callable<Integer>() {
    +            @Override
    +            public Integer call() throws Exception {
    +                //nimbus:num-blacklisted-supervisor + none blacklisted supervisor = nimbus:num-supervisors
    +                return blacklistHost.size();
    +            }
    +        });
    +    }
    +
    +    @Override
    +    public void schedule(Topologies topologies, Cluster cluster) {
    +        LOG.debug("running Black List scheduler");
    +        Map<String, SupervisorDetails> supervisors = cluster.getSupervisors();
    +        for (Map.Entry<String, SupervisorDetails> entry : supervisors.entrySet()) {
    +            SupervisorDetails supervisorDetails = entry.getValue();
    +            String hosts = supervisorDetails.getHost();
    +            Set<Integer> ports = supervisorDetails.getAllPorts();
    +            LOG.debug("supervisor: {}, ports: {}", hosts, ports);
    +        }
    +        LOG.debug("AssignableSlots: {}", cluster.getAssignableSlots());
    +        LOG.debug("AvailableSlots: {}", cluster.getAvailableSlots());
    +        LOG.debug("UsedSlots: {}", cluster.getUsedSlots());
    +
    +        blacklistStrategy.resumeFromBlacklist();
    +        badSupervisors(supervisors);
    +        cluster.setBlacklistedHosts(getBlacklistHosts(cluster, topologies));
    +        removeLongTimeDisappearFromCache();
    +
    +        underlyingScheduler.schedule(topologies, cluster);
    +    }
    +
    +    private void badSupervisors(Map<String, SupervisorDetails> supervisors) {
    +        Set<String> cachedSupervisorsKeySet = cachedSupervisors.keySet();
    +        Set<String> supervisorsKeySet = supervisors.keySet();
    +
    +        Set<String> badSupervisorKeys = Sets.difference(cachedSupervisorsKeySet, supervisorsKeySet);//cached supervisor doesn't show up
    +        HashMap<String, Set<Integer>> badSupervisors = new HashMap<String, Set<Integer>>();
    +        for (String key : badSupervisorKeys) {
    +            badSupervisors.put(key, cachedSupervisors.get(key));
    +        }
    +
    +        for (Map.Entry<String, SupervisorDetails> entry : supervisors.entrySet()) {
    +            String key = entry.getKey();
    +            SupervisorDetails supervisorDetails = entry.getValue();
    +            if (cachedSupervisors.containsKey(key)) {
    +                Set<Integer> badSlots = badSlots(supervisors, key);
    +                if (badSlots.size() > 0) {//supervisor contains bad slots
    +                    badSupervisors.put(key, badSlots);
    +                }
    +            } else {
    +                cachedSupervisors.put(key, supervisorDetails.getAllPorts());//new supervisor to cache
    +            }
    +        }
    +
    +        badSupervisorsTolerance.add(badSupervisors);
    +    }
    +
    +    private Set<Integer> badSlots(Map<String, SupervisorDetails> supervisors, String supervisorKey) {
    --- End diff --
    
    this should take SupervisorDetails directly instead of the supervisors map.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---