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

    https://github.com/apache/storm/pull/1642#discussion_r79036111
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java 
---
    @@ -0,0 +1,776 @@
    +/**
    + * 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.io.IOException;
    +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);
    +            sb.append(" msInState: ");
    +            sb.append(Time.currentTimeMillis() - startTime);
    +            if (container != null) {
    +                sb.append(" ");
    +                sb.append(container);
    +            }
    +            return sb.toString();
    +        }
    +
    +        public DynamicState withNewAssignment(LocalAssignment 
newAssignment) {
    +            return new DynamicState(this.state, newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    this.pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withPendingLocalization(LocalAssignment 
pendingLocalization, Future<Void> pendingDownload) {
    +            return new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withPendingLocalization(Future<Void> 
pendingDownload) {
    +            return withPendingLocalization(this.pendingLocalization, 
pendingDownload);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = Time.currentTimeMillis();
    +            return new DynamicState(state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, newStartTime,
    +                    this.pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +
    +        public DynamicState withCurrentAssignment(final Container 
container, final LocalAssignment currentAssignment) {
    +            return new DynamicState(this.state, this.newAssignment,
    +                    container, currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    this.pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +
    +        public DynamicState withProfileActions(Set<TopoProfileAction> 
profileActions, Set<TopoProfileAction> pendingStopProfileActions) {
    +            return new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    this.pendingDownload, profileActions,
    +                    pendingStopProfileActions);
    +        }
    +    };
    +    
    +    static class TopoProfileAction {
    +        public final String topoId;
    +        public final ProfileRequest request;
    +        
    +        public TopoProfileAction(String topoId, ProfileRequest request) {
    +            this.topoId = topoId;
    +            this.request = request;
    +        }
    +        
    +        @Override
    +        public int hashCode() {
    +            return (37 * topoId.hashCode()) + request.hashCode(); 
    +        }
    +        
    +        @Override
    +        public boolean equals(Object other) {
    +            if (!(other instanceof TopoProfileAction)) {
    +                return false;
    +            }
    +            TopoProfileAction o = (TopoProfileAction) other;
    +            return topoId.equals(o.topoId) && request.equals(o.request);
    +        }
    +        
    +        @Override
    +        public String toString() {
    +            return "{ " + topoId + ": " + request + " }";
    +        }
    +    }
    +    
    +    static boolean equivalent(LocalAssignment a, LocalAssignment b) {
    +        if (a == null && b == null) {
    +            return true;
    +        }
    +        if (a != null && b != null) {
    +            if (a.get_topology_id().equals(b.get_topology_id())) {
    +                Set<ExecutorInfo> aexec = new HashSet<>(a.get_executors());
    +                Set<ExecutorInfo> bexec = new HashSet<>(b.get_executors());
    +                if (aexec.equals(bexec)) {
    +                    boolean aHasResources = a.is_set_resources();
    +                    boolean bHasResources = b.is_set_resources();
    +                    if (!aHasResources && !bHasResources) {
    +                        return true;
    +                    }
    +                    if (aHasResources && bHasResources) {
    +                        if (a.get_resources().equals(b.get_resources())) {
    +                            return true;
    +                        }
    +                    }
    +                }
    +            }
    +        }
    +        return false;
    +    }
    +    
    +    static DynamicState stateMachineStep(DynamicState dynamicState, 
StaticState staticState) throws Exception {
    +        LOG.debug("STATE {}", dynamicState.state);
    +        switch (dynamicState.state) {
    +            case EMPTY:
    +                return handleEmpty(dynamicState, staticState);
    +            case RUNNING:
    +                return handleRunning(dynamicState, staticState);
    +            case WAITING_FOR_WORKER_START:
    +                return handleWaitingForWorkerStart(dynamicState, 
staticState);
    +            case KILL_AND_RELAUNCH:
    +                return handleKillAndRelaunch(dynamicState, staticState);
    +            case KILL:
    +                return handleKill(dynamicState, staticState);
    +            case WAITING_FOR_BASIC_LOCALIZATION:
    +                return handleWaitingForBasicLocalization(dynamicState, 
staticState);
    +            case WAITING_FOR_BLOB_LOCALIZATION:
    +                return handleWaitingForBlobLocalization(dynamicState, 
staticState);
    +            default:
    +                throw new IllegalStateException("Code not ready to handle 
a state of "+dynamicState.state);
    +        }
    +    }
    +    
    +    /**
    +     * Prepare for a new assignment my downloading new required blobs, or 
going to empty if there is nothing to download.
    +     * PRECONDITION: The slot should be empty
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws IOException on any error
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState 
dynamicState, StaticState staticState) throws IOException {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = 
staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment,
 staticState.port);
    +        return 
dynamicState.withPendingLocalization(dynamicState.newAssignment, 
pendingDownload).withState(MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +    }
    +    
    +    /**
    +     * Kill the current container and start downloading what the new 
assignment needs, if there is a new assignment
    +     * PRECONDITION: container != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception 
    +     */
    +    static DynamicState killContainerForChangedAssignment(DynamicState 
dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        
    +        staticState.iSupervisor.killedWorker(staticState.port);
    +        dynamicState.container.kill();
    +        Future<Void> pendingDownload = null;
    +        if (dynamicState.newAssignment != null) {
    +            pendingDownload = 
staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment,
 staticState.port);
    +        }
    +        Time.sleep(staticState.killSleepMs);
    +        return 
dynamicState.withPendingLocalization(dynamicState.newAssignment, 
pendingDownload).withState(MachineState.KILL);
    +    }
    +    
    +    /**
    +     * Kill the current container and relaunch it.  (Something odd 
happened)
    +     * PRECONDITION: container != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception 
    +     */
    +    static DynamicState killAndRelaunchContainer(DynamicState 
dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        
    +        dynamicState.container.kill();
    +        Time.sleep(staticState.killSleepMs);
    +        
    +        //any stop profile actions that hadn't timed out yet, we should 
restart after the worker is running again.
    +        HashSet<TopoProfileAction> mod = new 
HashSet<>(dynamicState.profileActions);
    +        mod.addAll(dynamicState.pendingStopProfileActions);
    +        return 
dynamicState.withState(MachineState.KILL_AND_RELAUNCH).withProfileActions(mod, 
Collections.<TopoProfileAction> emptySet());
    +    }
    +    
    +    /**
    +     * Clean up a container
    +     * PRECONDITION: All of the processes have died.
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @param nextState the next MachineState to go to.
    +     * @return the next state.
    +     */
    +    static DynamicState cleanupCurrentContainer(DynamicState dynamicState, 
StaticState staticState, MachineState nextState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        dynamicState.container.cleanUp();
    +        
staticState.localizer.releaseSlotFor(dynamicState.currentAssignment, 
staticState.port);
    +        DynamicState ret = dynamicState.withCurrentAssignment(null, null);
    +        if (nextState != null) {
    +            ret = ret.withState(nextState);
    +        }
    +        return ret;
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BLOB_LOCALIZATION state.
    +     * PRECONDITION: neither pendingLocalization nor pendingDownload is 
null.
    +     * PRECONDITION: The slot should be empty
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleWaitingForBlobLocalization(DynamicState 
dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    --- End diff --
    
    Right, sorry. I misunderstood the Javadoc. Should have paid more attention 
:)


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