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

    https://github.com/apache/storm/pull/1642#discussion_r77526779
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java 
---
    @@ -0,0 +1,766 @@
    +/**
    + * 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.supervisor;
    +
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class Slot extends Thread implements AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(Slot.class);
    +    
    +    static enum MachineState {
    +        EMPTY,
    +        RUNNING,
    +        WAITING_FOR_WORKER_START,
    +        KILL_AND_RELAUNCH,
    +        KILL,
    +        WAITING_FOR_BASIC_LOCALIZATION,
    +        WAITING_FOR_BLOB_LOCALIZATION;
    +    };
    +    
    +    static class StaticState {
    +        public final ILocalizer localizer;
    +        public final long hbTimeoutMs;
    +        public final long firstHbTimeoutMs;
    +        public final long killSleepMs;
    +        public final long monitorFreqMs;
    +        public final ContainerLauncher containerLauncher;
    +        public final int port;
    +        public final String host;
    +        public final ISupervisor iSupervisor;
    +        public final LocalState localState;
    +        
    +        StaticState(ILocalizer localizer, long hbTimeoutMs, long 
firstHbTimeoutMs,
    +                long killSleepMs, long monitorFreqMs,
    +                ContainerLauncher containerLauncher, String host, int port,
    +                ISupervisor iSupervisor, LocalState localState) {
    +            this.localizer = localizer;
    +            this.hbTimeoutMs = hbTimeoutMs;
    +            this.firstHbTimeoutMs = firstHbTimeoutMs;
    +            this.containerLauncher = containerLauncher;
    +            this.killSleepMs = killSleepMs;
    +            this.monitorFreqMs = monitorFreqMs;
    +            this.host = host;
    +            this.port = port;
    +            this.iSupervisor = iSupervisor;
    +            this.localState = localState;
    +        }
    +    }
    +    
    +    static class DynamicState {
    +        public final MachineState state;
    +        public final LocalAssignment newAssignment;
    +        public final LocalAssignment currentAssignment;
    +        public final Container container;
    +        public final LocalAssignment pendingLocalization;
    +        public final Future<Void> pendingDownload;
    +        public final Set<TopoProfileAction> profileActions;
    +        public final Set<TopoProfileAction> pendingStopProfileActions;
    +        
    +        /**
    +         * The last time that WAITING_FOR_WORKER_START, KILL, or 
KILL_AND_RELAUNCH were entered into.
    +         */
    +        public final long startTime;
    +        
    +        public DynamicState(final LocalAssignment currentAssignment, 
Container container, final LocalAssignment newAssignment) {
    +            this.currentAssignment = currentAssignment;
    +            this.container = container;
    +            if ((currentAssignment == null) ^ (container == null)) {
    +                throw new IllegalArgumentException("Container and current 
assignment must both be null, or neither can be null");
    +            }
    +            
    +            if (currentAssignment == null) {
    +                state = MachineState.EMPTY;
    +            } else {
    +                state = MachineState.RUNNING;
    +            }
    +            
    +            this.startTime = System.currentTimeMillis();
    +            this.newAssignment = newAssignment;
    +            this.pendingLocalization = null;
    +            this.pendingDownload = null;
    +            this.profileActions = new HashSet<>();
    +            this.pendingStopProfileActions = new HashSet<>();
    +        }
    +        
    +        public DynamicState(final MachineState state, final 
LocalAssignment newAssignment,
    +                final Container container, final LocalAssignment 
currentAssignment,
    +                final LocalAssignment pendingLocalization, final long 
startTime,
    +                final Future<Void> pendingDownload, final 
Set<TopoProfileAction> profileActions, 
    +                final Set<TopoProfileAction> pendingStopProfileActions) {
    +            this.state = state;
    +            this.newAssignment = newAssignment;
    +            this.currentAssignment = currentAssignment;
    +            this.container = container;
    +            this.pendingLocalization = pendingLocalization;
    +            this.startTime = startTime;
    +            this.pendingDownload = pendingDownload;
    +            this.profileActions = profileActions;
    +            this.pendingStopProfileActions = pendingStopProfileActions;
    +        }
    +        
    +        public String toString() {
    +            StringBuffer sb = new StringBuffer();
    +            sb.append(state);
    +            if (state == MachineState.WAITING_FOR_WORKER_START ||
    +                state == MachineState.KILL ||
    +                state == MachineState.KILL_AND_RELAUNCH) {
    +                sb.append(" msInState: ");
    +                sb.append(Time.currentTimeMillis() - startTime);
    +            }
    --- End diff --
    
    Originally I had it so that we would start the counter in one state and 
then time things out in a different state, but I combined states enough that we 
should be able to do it all the time.


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