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

    https://github.com/apache/storm/pull/1744#discussion_r90193679
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -0,0 +1,3729 @@
    +/**
    + * 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.daemon.nimbus;
    +
    +import static org.apache.storm.metric.StormMetricsRegistry.registerMeter;
    +import static org.apache.storm.utils.Utils.OR;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InterruptedIOException;
    +import java.io.OutputStream;
    +import java.net.BindException;
    +import java.net.ServerSocket;
    +import java.nio.ByteBuffer;
    +import java.nio.channels.Channels;
    +import java.nio.channels.WritableByteChannel;
    +import java.security.Principal;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.concurrent.atomic.AtomicReference;
    +import java.util.function.UnaryOperator;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +import javax.security.auth.Subject;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.StormTimer;
    +import org.apache.storm.blobstore.AtomicOutputStream;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.BlobStoreAclHandler;
    +import org.apache.storm.blobstore.BlobSynchronizer;
    +import org.apache.storm.blobstore.InputStreamWithMeta;
    +import org.apache.storm.blobstore.KeySequenceNumber;
    +import org.apache.storm.blobstore.LocalFsBlobStore;
    +import org.apache.storm.cluster.ClusterStateContext;
    +import org.apache.storm.cluster.ClusterUtils;
    +import org.apache.storm.cluster.DaemonType;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.daemon.DaemonCommon;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.StormCommon;
    +import org.apache.storm.generated.AlreadyAliveException;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.AuthorizationException;
    +import org.apache.storm.generated.BeginDownloadResult;
    +import org.apache.storm.generated.ClusterSummary;
    +import org.apache.storm.generated.CommonAggregateStats;
    +import org.apache.storm.generated.ComponentAggregateStats;
    +import org.apache.storm.generated.ComponentPageInfo;
    +import org.apache.storm.generated.ComponentType;
    +import org.apache.storm.generated.Credentials;
    +import org.apache.storm.generated.DebugOptions;
    +import org.apache.storm.generated.ErrorInfo;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.ExecutorStats;
    +import org.apache.storm.generated.ExecutorSummary;
    +import org.apache.storm.generated.GetInfoOptions;
    +import org.apache.storm.generated.InvalidTopologyException;
    +import org.apache.storm.generated.KeyAlreadyExistsException;
    +import org.apache.storm.generated.KeyNotFoundException;
    +import org.apache.storm.generated.KillOptions;
    +import org.apache.storm.generated.LSTopoHistory;
    +import org.apache.storm.generated.ListBlobsResult;
    +import org.apache.storm.generated.LogConfig;
    +import org.apache.storm.generated.LogLevel;
    +import org.apache.storm.generated.LogLevelAction;
    +import org.apache.storm.generated.Nimbus.Iface;
    +import org.apache.storm.generated.Nimbus.Processor;
    +import org.apache.storm.generated.NimbusSummary;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.NotAliveException;
    +import org.apache.storm.generated.NumErrorsChoice;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.ReadableBlobMeta;
    +import org.apache.storm.generated.RebalanceOptions;
    +import org.apache.storm.generated.SettableBlobMeta;
    +import org.apache.storm.generated.StormBase;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.SubmitOptions;
    +import org.apache.storm.generated.SupervisorInfo;
    +import org.apache.storm.generated.SupervisorPageInfo;
    +import org.apache.storm.generated.SupervisorSummary;
    +import org.apache.storm.generated.TopologyActionOptions;
    +import org.apache.storm.generated.TopologyHistoryInfo;
    +import org.apache.storm.generated.TopologyInfo;
    +import org.apache.storm.generated.TopologyInitialStatus;
    +import org.apache.storm.generated.TopologyPageInfo;
    +import org.apache.storm.generated.TopologyStatus;
    +import org.apache.storm.generated.TopologySummary;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.generated.WorkerSummary;
    +import org.apache.storm.logging.ThriftAccessLogger;
    +import org.apache.storm.metric.ClusterMetricsConsumerExecutor;
    +import org.apache.storm.metric.StormMetricsRegistry;
    +import org.apache.storm.metric.api.DataPoint;
    +import org.apache.storm.metric.api.IClusterMetricsConsumer;
    +import org.apache.storm.metric.api.IClusterMetricsConsumer.ClusterInfo;
    +import org.apache.storm.nimbus.DefaultTopologyValidator;
    +import org.apache.storm.nimbus.ILeaderElector;
    +import org.apache.storm.nimbus.ITopologyActionNotifierPlugin;
    +import org.apache.storm.nimbus.ITopologyValidator;
    +import org.apache.storm.nimbus.NimbusInfo;
    +import org.apache.storm.scheduler.Cluster;
    +import org.apache.storm.scheduler.DefaultScheduler;
    +import org.apache.storm.scheduler.ExecutorDetails;
    +import org.apache.storm.scheduler.INimbus;
    +import org.apache.storm.scheduler.IScheduler;
    +import org.apache.storm.scheduler.SchedulerAssignment;
    +import org.apache.storm.scheduler.SchedulerAssignmentImpl;
    +import org.apache.storm.scheduler.SupervisorDetails;
    +import org.apache.storm.scheduler.Topologies;
    +import org.apache.storm.scheduler.TopologyDetails;
    +import org.apache.storm.scheduler.WorkerSlot;
    +import org.apache.storm.scheduler.resource.ResourceUtils;
    +import org.apache.storm.security.INimbusCredentialPlugin;
    +import org.apache.storm.security.auth.AuthUtils;
    +import org.apache.storm.security.auth.IAuthorizer;
    +import org.apache.storm.security.auth.ICredentialsRenewer;
    +import org.apache.storm.security.auth.IGroupMappingServiceProvider;
    +import org.apache.storm.security.auth.IPrincipalToLocal;
    +import org.apache.storm.security.auth.NimbusPrincipal;
    +import org.apache.storm.security.auth.ReqContext;
    +import org.apache.storm.security.auth.ThriftConnectionType;
    +import org.apache.storm.security.auth.ThriftServer;
    +import org.apache.storm.stats.StatsUtil;
    +import org.apache.storm.utils.BufferInputStream;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.apache.storm.utils.TimeCacheMap;
    +import org.apache.storm.utils.TupleUtils;
    +import org.apache.storm.utils.Utils;
    +import org.apache.storm.utils.Utils.UptimeComputer;
    +import org.apache.storm.utils.VersionInfo;
    +import org.apache.storm.validation.ConfigValidation;
    +import org.apache.storm.zookeeper.Zookeeper;
    +import org.apache.thrift.TException;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.data.ACL;
    +import org.json.simple.JSONValue;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.codahale.metrics.Meter;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.collect.ImmutableMap;
    +
    +public class Nimbus implements Iface, Shutdownable, DaemonCommon {
    +    private final static Logger LOG = 
LoggerFactory.getLogger(Nimbus.class);
    +    
    +    //    Metrics
    +    private static final Meter submitTopologyWithOptsCalls = 
registerMeter("nimbus:num-submitTopologyWithOpts-calls");
    +    private static final Meter submitTopologyCalls = 
registerMeter("nimbus:num-submitTopology-calls");
    +    private static final Meter killTopologyWithOptsCalls = 
registerMeter("nimbus:num-killTopologyWithOpts-calls");
    +    private static final Meter killTopologyCalls = 
registerMeter("nimbus:num-killTopology-calls");
    +    private static final Meter rebalanceCalls = 
registerMeter("nimbus:num-rebalance-calls");
    +    private static final Meter activateCalls = 
registerMeter("nimbus:num-activate-calls");
    +    private static final Meter deactivateCalls = 
registerMeter("nimbus:num-deactivate-calls");
    +    private static final Meter debugCalls = 
registerMeter("nimbus:num-debug-calls");
    +    private static final Meter setWorkerProfilerCalls = 
registerMeter("nimbus:num-setWorkerProfiler-calls");
    +    private static final Meter getComponentPendingProfileActionsCalls = 
registerMeter("nimbus:num-getComponentPendingProfileActions-calls");
    +    private static final Meter setLogConfigCalls = 
registerMeter("nimbus:num-setLogConfig-calls");
    +    private static final Meter uploadNewCredentialsCalls = 
registerMeter("nimbus:num-uploadNewCredentials-calls");
    +    private static final Meter beginFileUploadCalls = 
registerMeter("nimbus:num-beginFileUpload-calls");
    +    private static final Meter uploadChunkCalls = 
registerMeter("nimbus:num-uploadChunk-calls");
    +    private static final Meter finishFileUploadCalls = 
registerMeter("nimbus:num-finishFileUpload-calls");
    +    private static final Meter beginFileDownloadCalls = 
registerMeter("nimbus:num-beginFileDownload-calls");
    +    private static final Meter downloadChunkCalls = 
registerMeter("nimbus:num-downloadChunk-calls");
    +    private static final Meter getNimbusConfCalls = 
registerMeter("nimbus:num-getNimbusConf-calls");
    +    private static final Meter getLogConfigCalls = 
registerMeter("nimbus:num-getLogConfig-calls");
    +    private static final Meter getTopologyConfCalls = 
registerMeter("nimbus:num-getTopologyConf-calls");
    +    private static final Meter getTopologyCalls = 
registerMeter("nimbus:num-getTopology-calls");
    +    private static final Meter getUserTopologyCalls = 
registerMeter("nimbus:num-getUserTopology-calls");
    +    private static final Meter getClusterInfoCalls = 
registerMeter("nimbus:num-getClusterInfo-calls");
    +    private static final Meter getTopologyInfoWithOptsCalls = 
registerMeter("nimbus:num-getTopologyInfoWithOpts-calls");
    +    private static final Meter getTopologyInfoCalls = 
registerMeter("nimbus:num-getTopologyInfo-calls");
    +    private static final Meter getTopologyPageInfoCalls = 
registerMeter("nimbus:num-getTopologyPageInfo-calls");
    +    private static final Meter getSupervisorPageInfoCalls = 
registerMeter("nimbus:num-getSupervisorPageInfo-calls");
    +    private static final Meter getComponentPageInfoCalls = 
registerMeter("nimbus:num-getComponentPageInfo-calls");
    +    private static final Meter shutdownCalls = 
registerMeter("nimbus:num-shutdown-calls");
    +    // END Metrics
    +    
    +    private static final String STORM_VERSION = VersionInfo.getVersion();
    +    @VisibleForTesting
    +    public static final List<ACL> ZK_ACLS = 
Arrays.asList(ZooDefs.Ids.CREATOR_ALL_ACL.get(0),
    +            new ACL(ZooDefs.Perms.READ | ZooDefs.Perms.CREATE, 
ZooDefs.Ids.ANYONE_ID_UNSAFE));
    +    private static final Subject NIMBUS_SUBJECT = new Subject();
    +    static {
    +        NIMBUS_SUBJECT.getPrincipals().add(new NimbusPrincipal());
    +        NIMBUS_SUBJECT.setReadOnly();
    +    }
    +    
    +    // TOPOLOGY STATE TRANSITIONS
    +    private static StormBase make(TopologyStatus status) {
    +        StormBase ret = new StormBase();
    +        ret.set_status(status);
    +        //The following are required for backwards compatibility with 
clojure code
    +        ret.set_component_executors(Collections.emptyMap());
    +        ret.set_component_debug(Collections.emptyMap());
    +        return ret;
    +    }
    +    
    +    private static final TopologyStateTransition NOOP_TRANSITION = (arg, 
nimbus, topoId, base) -> null;
    +    private static final TopologyStateTransition INACTIVE_TRANSITION = 
(arg, nimbus, topoId, base) -> Nimbus.make(TopologyStatus.INACTIVE);
    +    private static final TopologyStateTransition ACTIVE_TRANSITION = (arg, 
nimbus, topoId, base) -> Nimbus.make(TopologyStatus.ACTIVE);
    +    private static final TopologyStateTransition KILL_TRANSITION = 
(killTime, nimbus, topoId, base) -> {
    +        int delay = 0;
    +        if (killTime != null) {
    +            delay = ((Number)killTime).intValue();
    +        } else {
    +            delay = Utils.getInt(Nimbus.readTopoConf(topoId, 
nimbus.getBlobStore()).get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS));
    +        }
    +        nimbus.delayEvent(topoId, delay, TopologyActions.REMOVE, null);
    +        StormBase sb = new StormBase();
    +        sb.set_status(TopologyStatus.KILLED);
    +        TopologyActionOptions tao = new TopologyActionOptions();
    +        KillOptions opts = new KillOptions();
    +        opts.set_wait_secs(delay);
    +        tao.set_kill_options(opts);
    +        sb.set_topology_action_options(tao);
    +        sb.set_component_executors(Collections.emptyMap());
    +        sb.set_component_debug(Collections.emptyMap());
    +        return sb;
    +    };
    +    
    +    private static final TopologyStateTransition REBALANCE_TRANSITION = 
(args, nimbus, topoId, base) -> {
    +        RebalanceOptions rbo = ((RebalanceOptions) args).deepCopy();
    +        int delay = 0;
    +        if (rbo.is_set_wait_secs()) {
    +            delay = rbo.get_wait_secs();
    +        } else {
    +            delay = Utils.getInt(Nimbus.readTopoConf(topoId, 
nimbus.getBlobStore()).get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS));
    +        }
    +        nimbus.delayEvent(topoId, delay, TopologyActions.DO_REBALANCE, 
null);
    +        
    +        rbo.set_wait_secs(delay);
    +        if (!rbo.is_set_num_executors()) {
    +            rbo.set_num_executors(Collections.emptyMap());
    +        }
    +        
    +        StormBase sb = new StormBase();
    +        sb.set_status(TopologyStatus.REBALANCING);
    +        sb.set_prev_status(base.get_status());
    +        TopologyActionOptions tao = new TopologyActionOptions();
    +        tao.set_rebalance_options(rbo);
    +        sb.set_topology_action_options(tao);
    +        sb.set_component_executors(Collections.emptyMap());
    +        sb.set_component_debug(Collections.emptyMap());
    +        
    +        return sb;
    +    };
    +    
    +    private static final TopologyStateTransition 
STARTUP_WHEN_KILLED_TRANSITION = (args, nimbus, topoId, base) -> {
    +        int delay = 
base.get_topology_action_options().get_kill_options().get_wait_secs();
    +        nimbus.delayEvent(topoId, delay, TopologyActions.REMOVE, null);
    +        return null;
    +    };
    +    
    +    private static final TopologyStateTransition REMOVE_TRANSITION = 
(args, nimbus, topoId, base) -> {
    +        LOG.info("Killing topology: {}", topoId);
    +        IStormClusterState state = nimbus.getStormClusterState();
    +        state.removeStorm(topoId);
    +        BlobStore store = nimbus.getBlobStore();
    +        if (store instanceof LocalFsBlobStore) {
    +            for (String key: Nimbus.getKeyListFromId(nimbus.getConf(), 
topoId)) {
    +                state.removeBlobstoreKey(key);
    +                state.removeKeyVersion(key);
    +            }
    +        }
    +        return null;
    +    };
    +    
    +    private static final TopologyStateTransition 
STARTUP_WHEN_REBALANCING_TRANSITION = (args, nimbus, topoId, base) -> {
    +        int delay = 
base.get_topology_action_options().get_rebalance_options().get_wait_secs();
    +        nimbus.delayEvent(topoId, delay, TopologyActions.DO_REBALANCE, 
null);
    +        return null;
    +    };
    +    
    +    private static final TopologyStateTransition DO_REBALANCE_TRANSITION = 
(args, nimbus, topoId, base) -> {
    +        nimbus.doRebalance(topoId, base);
    +        return Nimbus.make(base.get_prev_status());
    +    };
    +    
    +    private static final Map<TopologyStatus, Map<TopologyActions, 
TopologyStateTransition>> TOPO_STATE_TRANSITIONS = 
    +            new ImmutableMap.Builder<TopologyStatus, Map<TopologyActions, 
TopologyStateTransition>>()
    +            .put(TopologyStatus.ACTIVE, new 
ImmutableMap.Builder<TopologyActions, TopologyStateTransition>()
    +                    .put(TopologyActions.INACTIVATE, INACTIVE_TRANSITION)
    +                    .put(TopologyActions.ACTIVATE, NOOP_TRANSITION)
    +                    .put(TopologyActions.REBALANCE, REBALANCE_TRANSITION)
    +                    .put(TopologyActions.KILL, KILL_TRANSITION)
    +                    .build())
    +            .put(TopologyStatus.INACTIVE, new 
ImmutableMap.Builder<TopologyActions, TopologyStateTransition>()
    +                    .put(TopologyActions.ACTIVATE, ACTIVE_TRANSITION)
    +                    .put(TopologyActions.INACTIVATE, NOOP_TRANSITION)
    +                    .put(TopologyActions.REBALANCE, REBALANCE_TRANSITION)
    +                    .put(TopologyActions.KILL, KILL_TRANSITION)
    +                    .build())
    +            .put(TopologyStatus.KILLED, new 
ImmutableMap.Builder<TopologyActions, TopologyStateTransition>()
    +                    .put(TopologyActions.STARTUP, 
STARTUP_WHEN_KILLED_TRANSITION)
    +                    .put(TopologyActions.KILL, KILL_TRANSITION)
    +                    .put(TopologyActions.REMOVE, REMOVE_TRANSITION)
    +                    .build())
    +            .put(TopologyStatus.REBALANCING, new 
ImmutableMap.Builder<TopologyActions, TopologyStateTransition>()
    +                    .put(TopologyActions.STARTUP, 
STARTUP_WHEN_REBALANCING_TRANSITION)
    +                    .put(TopologyActions.KILL, KILL_TRANSITION)
    +                    .put(TopologyActions.DO_REBALANCE, 
DO_REBALANCE_TRANSITION)
    +                    .build())
    +            .build();
    +    
    +    // END TOPOLOGY STATE TRANSITIONS
    +    
    +    private static final class Assoc<K,V> implements UnaryOperator<Map<K, 
V>> {
    +        private final K key;
    +        private final V value;
    +        
    +        public Assoc(K key, V value) {
    +            this.key = key;
    +            this.value = value;
    +        }
    +        
    +        @Override
    +        public Map<K, V> apply(Map<K, V> t) {
    +            Map<K, V> ret = new HashMap<>(t);
    +            ret.put(key, value);
    +            return ret;
    +        }
    +    }
    +    
    +    private static final class Dissoc<K,V> implements UnaryOperator<Map<K, 
V>> {
    +        private final K key;
    +        
    +        public Dissoc(K key) {
    +            this.key = key;
    +        }
    +        
    +        @Override
    +        public Map<K, V> apply(Map<K, V> t) {
    +            Map<K, V> ret = new HashMap<>(t);
    +            ret.remove(key);
    +            return ret;
    +        }
    +    }
    +    
    +    @VisibleForTesting
    +    public static class StandAloneINimbus implements INimbus {
    +
    +        @Override
    +        public void prepare(@SuppressWarnings("rawtypes") Map stormConf, 
String schedulerLocalDir) {
    +            //NOOP
    +        }
    +
    +        @SuppressWarnings("unchecked")
    +        @Override
    +        public Collection<WorkerSlot> 
allSlotsAvailableForScheduling(Collection<SupervisorDetails> supervisors,
    +                Topologies topologies, Set<String> 
topologiesMissingAssignments) {
    +            Set<WorkerSlot> ret = new HashSet<>();
    +            for (SupervisorDetails sd: supervisors) {
    +                String id = sd.getId();
    +                for (Number port: (Collection<Number>)sd.getMeta()) {
    +                    ret.add(new WorkerSlot(id, port));
    +                }
    +            }
    +            return ret;
    +        }
    +
    +        @Override
    +        public void assignSlots(Topologies topologies, Map<String, 
Collection<WorkerSlot>> newSlotsByTopologyId) {
    +            //NOOP
    +        }
    +
    +        @Override
    +        public String getHostName(Map<String, SupervisorDetails> 
supervisors, String nodeId) {
    +            SupervisorDetails sd = supervisors.get(nodeId);
    +            if (sd != null) {
    +                return sd.getHost();
    +            }
    +            return null;
    +        }
    +
    +        @Override
    +        public IScheduler getForcedScheduler() {
    +            return null;
    +        }
    +        
    +    };
    +    
    +    private static class CommonTopoInfo {
    +        public Map<String, Object> topoConf;
    +        public String topoName;
    +        public StormTopology topology;
    +        public Map<Integer, String> taskToComponent;
    +        public StormBase base;
    +        public int launchTimeSecs;
    +        public Assignment assignment;
    +        public Map<List<Integer>, Map<String, Object>> beats;
    +        public HashSet<String> allComponents;
    +
    +    }
    +    
    +    @SuppressWarnings("deprecation")
    +    private static <T extends AutoCloseable> TimeCacheMap<String, T> 
fileCacheMap(Map<String, Object> conf) {
    +        return new 
TimeCacheMap<>(Utils.getInt(conf.get(Config.NIMBUS_FILE_COPY_EXPIRATION_SECS), 
600),
    +                (id, stream) -> {
    +                    try {
    +                        stream.close();
    +                    } catch (Exception e) {
    +                        throw new RuntimeException(e);
    +                    }
    +                });
    +    }
    +
    +    private static <K, V> Map<K, V> merge(Map<? extends K, ? extends V> 
first, Map<? extends K, ? extends V> other) {
    +        Map<K, V> ret = new HashMap<>(first);
    +        if (other != null) {
    +            ret.putAll(other);
    +        }
    +        return ret;
    +    }
    +    
    +    private static <K, V> Map<K, V> mapDiff(Map<? extends K, ? extends V> 
first, Map<? extends K, ? extends V> second) {
    +        Map<K, V> ret = new HashMap<>();
    +        for (Entry<? extends K, ? extends V> entry: second.entrySet()) {
    +            if (!entry.getValue().equals(first.get(entry.getKey()))) {
    +                ret.put(entry.getKey(), entry.getValue());
    +            }
    +        }
    +        return ret;
    +    }
    +
    +    private static IScheduler makeScheduler(Map<String, Object> conf, 
INimbus inimbus) {
    +        String schedClass = (String) conf.get(Config.STORM_SCHEDULER);
    +        IScheduler scheduler = inimbus == null ? null : 
inimbus.getForcedScheduler();
    +        if (scheduler != null) {
    +            LOG.info("Using forced scheduler from INimbus {} {}", 
scheduler.getClass(), scheduler);
    +        } else if (schedClass != null){
    --- End diff --
    
    nit: space between `)` and `{`  (Yes, I'm annoying like this 😉)


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to