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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

GitHub user revans2 opened a pull request:

    https://github.com/apache/storm/pull/1642

    DO NOT MERGE:  Please review STORM-2018: Supervisor V2.  

    Still needs run as user and CGroup work, but the rest is working
    
    Any feedback on this would be welcome.  I am particularly interested in the readability of the code and how easy is it to understand vs the original supervisor.
    
    All the unit tests pass and I have done a lot of manual testing switching back and forth between the original supervisor and supervisor V2 to ensure that this can be a rolling upgrade.
    
    There are still a number of TODOs in the code.  Most of them are for removing the original supervisor code and cleaning up the result of that.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/revans2/incubator-storm STORM-2018

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/storm/pull/1642.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1642
    
----
commit 33483890e8eb32497a6ca7073395889044402d21
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Date:   2016-08-05T21:15:21Z

    STORM-2018: Supervisor V2.  Still needs runas user and CGRoup work, but the rest is working

----


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76271101
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,494 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +    
    +    private class ProcessExitCallback implements Utils.ExitCodeCallable<Void> {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public Void call() throws Exception {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +            return null;
    +        }
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, LocalState localState, 
    +            ResourceIsolationInterface resourceIsolationManager, boolean recover) throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized(localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry: workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port +" "+ assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +        
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +    
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized(_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +        synchronized(_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            _localState.setApprovedWorkers(workersToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +    
    +    /**
    +     * Run the given command for profiling
    +     * @param command the command to run
    +     * @param env the environment to run the command
    +     * @param logPrefix the prefix to include in the logs
    +     * @param targetDir the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException on any error
    +     * @throws InterruptedException if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix, File targetDir) throws IOException, InterruptedException {
    +        Process p = Utils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +    
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String topologyId = _assignment.get_topology_id();
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, topologyId, _port);
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) topologyConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, topologyId, _port);
    +
    +        String workerPid = null;
    +        try (FileReader reader = new FileReader(str); BufferedReader br = new BufferedReader(reader)) {
    +            workerPid = br.readLine().trim();
    +        }
    +        
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        return runProfilingCommand(command, env, logPrefix, targetFile);
    +    }
    +    
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        if (action == ProfileAction.JMAP_DUMP) {
    +            return jmapDumpCmd(workerPid, targetDir);
    +        } else if (action == ProfileAction.JSTACK_DUMP) {
    +            return jstackDumpCmd(workerPid, targetDir);
    +        } else if (action == ProfileAction.JPROFILE_DUMP) {
    +            return jprofileDump(workerPid, targetDir);
    +        } else if (action == ProfileAction.JVM_RESTART) {
    +            return jprofileJvmRestart(workerPid);
    +        } else if (!stop && action == ProfileAction.JPROFILE_STOP) {
    +            return jprofileStart(workerPid);
    +        } else if (stop && action == ProfileAction.JPROFILE_STOP) {
    +            return jprofileStop(workerPid, targetDir);
    +        }
    +        return Lists.newArrayList();
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +    
    +    protected String jlp(String stormRoot, Map<String, Object> conf) {
    +        String resourceRoot = stormRoot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +        String os = System.getProperty("os.name").replaceAll("\\s+", "_");
    +        String arch = System.getProperty("os.arch");
    +        String archResourceRoot = resourceRoot + Utils.FILE_PATH_SEPARATOR + os + "-" + arch;
    +        String ret = archResourceRoot + Utils.CLASS_PATH_SEPARATOR + resourceRoot + Utils.CLASS_PATH_SEPARATOR + conf.get(Config.JAVA_LIBRARY_PATH);
    +        return ret;
    +    }
    +    
    +    @SuppressWarnings("unchecked")
    +    protected String getWorkerClassPath(String stormJar, Map<String, Object> topologyConf) {
    +        List<String> topoClasspath = new ArrayList<>();
    +        Object object = topologyConf.get(Config.TOPOLOGY_CLASSPATH);
    +
    +        // Will be populated only if STORM_TOPOLOGY_CLASSPATH_BEGINNING_ENABLED is set on Nimbus.
    +        // Allowed for extreme debugging.
    +        Object topologyClasspathFirst = topologyConf.get(Config.TOPOLOGY_CLASSPATH_BEGINNING);
    +        List<String> firstClasspathList = new ArrayList<>();
    +        if(topologyClasspathFirst instanceof List) {
    +            firstClasspathList.addAll((List<String>)topologyClasspathFirst);
    +        } else if (topologyClasspathFirst instanceof String) {
    +            firstClasspathList.add((String) topologyClasspathFirst);
    +        }
    +        LOG.debug("Topology Classpath Prefix: {}", firstClasspathList);
    +
    +        if (object instanceof List) {
    +            topoClasspath.addAll((List<String>) object);
    +        } else if (object instanceof String) {
    +            topoClasspath.add((String) object);
    +        }
    +        LOG.debug("Topology specific classpath is {}", object);
    +
    +        String classPath = Utils.addToClasspath(firstClasspathList, Arrays.asList(Utils.workerClasspath()));
    +        String classAddPath = Utils.addToClasspath(classPath, Arrays.asList(stormJar));
    +        return Utils.addToClasspath(classAddPath, topoClasspath);
    +    }
    +    
    +    private static String substituteChildOptsInternal(String string,  String workerId, String stormId, int port, int memOnheap) {
    +        if (StringUtils.isNotBlank(string)){
    +            string = string.replace("%ID%", String.valueOf(port));
    +            string = string.replace("%WORKER-ID%", workerId);
    +            string = string.replace("%TOPOLOGY-ID%", stormId);
    +            string = string.replace("%WORKER-PORT%", String.valueOf(port));
    +            string = string.replace("%HEAP-MEM%", String.valueOf(memOnheap));
    +        }
    +        return string;
    +    }
    +    
    +    protected List<String> substituteChildopts(Object value, String workerId, String stormId, int port, int memOnheap) {
    +        List<String> rets = new ArrayList<>();
    +        if (value instanceof String) {
    +            String string = substituteChildOptsInternal((String) value,  workerId, stormId, port, memOnheap);
    +            if (StringUtils.isNotBlank(string)){
    +                String[] strings = string.split("\\s+");
    +                rets.addAll(Arrays.asList(strings));
    +            }
    +        } else if (value instanceof List) {
    +            @SuppressWarnings("unchecked")
    +            List<Object> objects = (List<Object>) value;
    +            for (Object object : objects) {
    +                String str = substituteChildOptsInternal((String) object,  workerId, stormId, port, memOnheap);
    +                if (StringUtils.isNotBlank(str)){
    +                    rets.add(str);
    +                }
    +            }
    +        }
    +        return rets;
    +    }
    +
    +    /**
    +     * Launch the worker process (non-blocking)
    +     * @param command the command to run
    +     * @param env the environment to run the command
    +     * @param processExitcallback a callback for when the process exits
    +     * @param logPrefix the prefix to include in the logs
    +     * @param targetDir the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException on any error
    +     */
    +    protected void launchWorkerProcess(List<String> command, Map<String, String> env, 
    +            String logPrefix, Utils.ExitCodeCallable<Void> processExitCallback, File targetDir) throws IOException {
    +        Utils.launchProcess(command, env, logPrefix, processExitCallback, targetDir);
    +    }
    +    
    +    @Override
    +    public void launch() throws IOException {
    --- End diff --
    
    Also It will be greatly helpful if we can break this into multiple methods.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76474776
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,437 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = -1;
    +        _assignment = null;
    +        _workerId = workerId;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq"+pid);
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            if (in.readLine() != null) {
    +                ret = true;
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    private boolean isPosixProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("ps", "-o", "user", "-p", String.valueOf(pid));
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            String first = in.readLine();
    +            assert("USER".equals(first));
    +            String processUser;
    +            while ((processUser = in.readLine()) != null) {
    +                if (user.equals(processUser)) {
    +                    ret = true;
    +                    break;
    +                } else {
    +                    LOG.info("Found {} running as {}, but expected it to be {}", pid, processUser, user);
    +                }
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    @Override
    +    public boolean areAllProcessesDead() throws IOException {
    +        Set<Long> pids = getAllPids();
    +        String user = getWorkerUser();
    +        
    +        boolean allDead = true;
    +        for (Long pid: pids) {
    +            if (!isProcessAlive(pid, user)) {
    +                LOG.warn("{}: PID {} is dead", _workerId, pid);
    +            } else {
    +                allDead = false;
    +                break;
    +            }
    +        }
    +        return allDead;
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +    }
    +
    +    /**
    +     * Setup the container to run.  By default this creates the needed directories/links in the
    +     * local file system
    +     * PREREQUISITE: All needed blobs and topology, jars/configs have been downloaded and
    +     * placed in the appropriate locations
    +     * @throws IOException on any error
    +     */
    +    protected void setup() throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    +        }
    +        final String topologyId = _assignment.get_topology_id();
    +        if (!SupervisorUtils.doRequiredTopoFilesExist(_conf, topologyId)) {
    +            LOG.info("Missing topology storm code, so can't launch  worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                    _supervisorId, _port, _workerId);
    --- End diff --
    
    LOG.error, and maybe just say missing required files? We don't know specifically what was missing.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76460236
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/ResourceIsolationInterface.java ---
    @@ -56,4 +64,13 @@
          */
         List<String> getLaunchCommandPrefix(String workerId);
     
    +    /**
    +     * Get the list of PIDs currently in an isolated container
    +     * @param workerId the id of the worker to get these for
    +     * @return the set of PIDs, this will be combined with
    +     * other ways of getting PIDs. An Empty set or null if
    +     * no PIDs are found.
    +     * @throws IOException on any error
    +     */
    --- End diff --
    
    Let's just return a Set and have it be empty if no PIDs are found. Callers then don't have to check both cases.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76800917
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,629 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +
    +    //For testing purposes
    +    public BasicContainer(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager, LocalState localState,
    +            String profileCmd) throws IOException {
    +        super(ops, port, assignment, conf, topoConf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +        _profileCmd = profileCmd;
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState localState, ResourceIsolationInterface resourceIsolationManager, boolean recover)
    +            throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException(
    +                    "Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            _localState.setApprovedWorkers(workersToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        setup();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = _ops.slurpString(new File(str)).trim();
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        if (command.size() > 0) {
    +            return runProfilingCommand(command, env, logPrefix, targetFile);
    +        }
    +        LOG.warn("PROFILING REQUEST NOT SUPPORTED {} IGNORED...", request);
    +        return true;
    +    }
    +
    +    /**
    +     * Get the command to run when doing profiling
    +     * @param action the profiling action to perform
    +     * @param stop if this is meant to stop the profiling or start it
    +     * @param workerPid the PID of the process to profile
    +     * @param targetDir the current working directory of the worker process
    +     * @return the command to run for profiling.
    +     */
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        switch(action) {
    +            case JMAP_DUMP:
    +                return jmapDumpCmd(workerPid, targetDir);
    +            case JSTACK_DUMP:
    +                return jstackDumpCmd(workerPid, targetDir);
    +            case JPROFILE_DUMP:
    +                return jprofileDump(workerPid, targetDir);
    +            case JVM_RESTART:
    +                return jprofileJvmRestart(workerPid);
    +            case JPROFILE_STOP:
    +                if (stop) {
    +                    return jprofileStop(workerPid, targetDir);
    +                }
    +                return jprofileStart(workerPid);
    +            default:
    +                return Lists.newArrayList();
    +        }
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +
    +    /**
    +     * Compute the java.library.path that should be used for the worker.
    +     * This helps it to load JNI libraries that are packaged in the uber jar.
    +     * @param stormRoot the root directory of the worker process
    +     * @param conf the config for the supervisor.
    +     * @return the java.library.path/LD_LIBRARY_PATH to use so native libraries load correctly.
    +     */
    +    protected String jlp(String stormRoot, Map<String, Object> conf) {
    +        String resourceRoot = stormRoot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +        String os = System.getProperty("os.name").replaceAll("\\s+", "_");
    +        String arch = System.getProperty("os.arch");
    +        String archResourceRoot = resourceRoot + Utils.FILE_PATH_SEPARATOR + os + "-" + arch;
    +        String ret = CPJ.join(archResourceRoot, resourceRoot,
    +                conf.get(Config.JAVA_LIBRARY_PATH));
    +        return ret;
    +    }
    +
    +    /**
    +     * Returns a collection of jar file names found under the given directory.
    +     * @param dir the directory to search
    +     * @return the jar file names
    +     */
    +    protected List<String> getFullJars(File dir) {
    +        File[] files = dir.listFiles(jarFilter);
    +
    +        if (files == null) {
    +            return Collections.emptyList();
    +        }
    +
    +        return Arrays.stream(files).map(f -> f.getAbsolutePath())
    +                .collect(Collectors.toList());
    +    }
    +    
    +    protected List<String> frameworkClasspath() {
    +        String stormHome = System.getProperty("storm.home");
    +
    +        File stormLibDir = new File(stormHome, "lib");
    +        String stormConfDir =
    +                System.getenv("STORM_CONF_DIR") != null ?
    +                System.getenv("STORM_CONF_DIR") :
    +                new File(stormHome, "conf").getAbsolutePath();
    +        File stormExtlibDir = new File(stormHome, "extlib");
    +        String extcp = System.getenv("STORM_EXT_CLASSPATH");
    +        List<String> pathElements = new LinkedList<>();
    +        pathElements.addAll(getFullJars(stormLibDir));
    +        pathElements.addAll(getFullJars(stormExtlibDir));
    +        pathElements.add(extcp);
    +        pathElements.add(stormConfDir);
    +
    +        return pathElements;
    +    }
    +    
    +    @SuppressWarnings("unchecked")
    +    private List<String> asStringList(Object o) {
    +        if (o instanceof String) {
    +            return Arrays.asList((String)o);
    +        } else if (o instanceof List) {
    +            return (List<String>)o;
    +        }
    +        return Collections.EMPTY_LIST;
    +    }
    +    
    +    /**
    +     * Compute the classpath for the worker process
    +     * @param stormJar the topology jar
    +     * @param dependencyLocations any dependencies from the topology
    +     * @return the full classpath
    +     */
    +    protected String getWorkerClassPath(String stormJar, List<String> dependencyLocations) {
    +        List<String> workercp = new ArrayList<>();
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH_BEGINNING)));
    +        workercp.addAll(frameworkClasspath());
    +        workercp.add(stormJar);
    +        workercp.addAll(dependencyLocations);
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH)));
    +        return CPJ.join(workercp);
    +    }
    +
    +    private String substituteChildOptsInternal(String string, int memOnheap) {
    +        if (StringUtils.isNotBlank(string)) {
    +            String p = String.valueOf(_port);
    +            string = string.replace("%ID%", p);
    +            string = string.replace("%WORKER-ID%", _workerId);
    +            string = string.replace("%TOPOLOGY-ID%", _topologyId);
    +            string = string.replace("%WORKER-PORT%", p);
    +            if (memOnheap > 0) {
    +                string = string.replace("%HEAP-MEM%", String.valueOf(memOnheap));
    +            }
    +        }
    +        return string;
    +    }
    +    
    +    protected List<String> substituteChildopts(Object value) {
    +        return substituteChildopts(value, -1);
    +    }
    +
    +    protected List<String> substituteChildopts(Object value, int memOnheap) {
    +        List<String> rets = new ArrayList<>();
    +        if (value instanceof String) {
    +            String string = substituteChildOptsInternal((String) value, memOnheap);
    +            if (StringUtils.isNotBlank(string)) {
    +                String[] strings = string.split("\\s+");
    +                rets.addAll(Arrays.asList(strings));
    +            }
    +        } else if (value instanceof List) {
    +            @SuppressWarnings("unchecked")
    +            List<String> objects = (List<String>) value;
    +            for (String object : objects) {
    +                String str = substituteChildOptsInternal(object, memOnheap);
    +                if (StringUtils.isNotBlank(str)) {
    +                    rets.add(str);
    +                }
    +            }
    +        }
    +        return rets;
    +    }
    +
    +    /**
    +     * Launch the worker process (non-blocking)
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param processExitcallback
    +     *            a callback for when the process exits
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     */
    +    protected void launchWorkerProcess(List<String> command, Map<String, String> env, String logPrefix,
    +            ExitCodeCallback processExitCallback, File targetDir) throws IOException {
    +        SupervisorUtils.launchProcess(command, env, logPrefix, processExitCallback, targetDir);
    +    }
    +
    +    private String getWorkerLoggingConfigFile(String stormHome) {
    +        String log4jConfigurationDir = (String) (_conf.get(Config.STORM_LOG4J2_CONF_DIR));
    +
    +        if (StringUtils.isNotBlank(log4jConfigurationDir)) {
    +            if (!Utils.isAbsolutePath(log4jConfigurationDir)) {
    +                log4jConfigurationDir = stormHome + Utils.FILE_PATH_SEPARATOR + log4jConfigurationDir;
    +            }
    +        } else {
    +            log4jConfigurationDir = stormHome + Utils.FILE_PATH_SEPARATOR + "log4j2";
    +        }
    +        
    +
    --- End diff --
    
    nit, blank line


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76477726
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,437 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = -1;
    +        _assignment = null;
    +        _workerId = workerId;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq"+pid);
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            if (in.readLine() != null) {
    +                ret = true;
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    private boolean isPosixProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("ps", "-o", "user", "-p", String.valueOf(pid));
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            String first = in.readLine();
    +            assert("USER".equals(first));
    +            String processUser;
    +            while ((processUser = in.readLine()) != null) {
    +                if (user.equals(processUser)) {
    +                    ret = true;
    +                    break;
    +                } else {
    +                    LOG.info("Found {} running as {}, but expected it to be {}", pid, processUser, user);
    +                }
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    @Override
    +    public boolean areAllProcessesDead() throws IOException {
    +        Set<Long> pids = getAllPids();
    +        String user = getWorkerUser();
    +        
    +        boolean allDead = true;
    +        for (Long pid: pids) {
    +            if (!isProcessAlive(pid, user)) {
    +                LOG.warn("{}: PID {} is dead", _workerId, pid);
    +            } else {
    +                allDead = false;
    +                break;
    +            }
    +        }
    +        return allDead;
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +    }
    +
    +    /**
    +     * Setup the container to run.  By default this creates the needed directories/links in the
    +     * local file system
    +     * PREREQUISITE: All needed blobs and topology, jars/configs have been downloaded and
    +     * placed in the appropriate locations
    +     * @throws IOException on any error
    +     */
    +    protected void setup() throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    +        }
    +        final String topologyId = _assignment.get_topology_id();
    +        if (!SupervisorUtils.doRequiredTopoFilesExist(_conf, topologyId)) {
    +            LOG.info("Missing topology storm code, so can't launch  worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                    _supervisorId, _port, _workerId);
    +            throw new IllegalStateException("Not all needed files are here!!!!");
    +        }
    +        String pidsPath = ConfigUtils.workerPidsRoot(_conf, _workerId);
    +        String hbPath = ConfigUtils.workerHeartbeatsRoot(_conf, _workerId);
    +    
    +        FileUtils.forceMkdir(new File(pidsPath));
    +        FileUtils.forceMkdir(new File(ConfigUtils.workerTmpRoot(_conf, _workerId)));
    +        FileUtils.forceMkdir(new File(hbPath));
    +    
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        String user = (String) topologyConf.get(Config.TOPOLOGY_SUBMITTER_USER);
    +        writeLogMetadata(topologyConf, user, topologyId);
    +        ConfigUtils.setWorkerUserWSE(_conf, _workerId, user);
    +        createArtifactsLink(topologyId);
    +    
    +        createBlobstoreLinks(topologyId);
    +    }
    +    
    +    /**
    +     * Write out the file used by the log viewer to allow/reject log access
    +     * @param topologyConf the config for the topology
    +     * @param user the user this is going to run as
    +     * @param topologyId the id of the topology
    +     * @throws IOException on any error
    +     */
    +    @SuppressWarnings("unchecked")
    +    protected void writeLogMetadata(Map<String, Object> topologyConf, String user, String topologyId) throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    +        }
    +        Map<String, Object> data = new HashMap<>();
    +        data.put(Config.TOPOLOGY_SUBMITTER_USER, user);
    +        data.put("worker-id", _workerId);
    +
    +        Set<String> logsGroups = new HashSet<>();
    +        //for supervisor-test
    +        if (topologyConf.get(Config.LOGS_GROUPS) != null) {
    +            List<String> groups = (List<String>) topologyConf.get(Config.LOGS_GROUPS);
    +            for (String group : groups){
    +                logsGroups.add(group);
    +            }
    +        }
    +        if (topologyConf.get(Config.TOPOLOGY_GROUPS) != null) {
    +            List<String> topGroups = (List<String>) topologyConf.get(Config.TOPOLOGY_GROUPS);
    +            logsGroups.addAll(topGroups);
    +        }
    +        data.put(Config.LOGS_GROUPS, logsGroups.toArray());
    +
    +        Set<String> logsUsers = new HashSet<>();
    +        if (topologyConf.get(Config.LOGS_USERS) != null) {
    +            List<String> logUsers = (List<String>) topologyConf.get(Config.LOGS_USERS);
    +            for (String logUser : logUsers){
    +                logsUsers.add(logUser);
    +            }
    +        }
    +        if (topologyConf.get(Config.TOPOLOGY_USERS) != null) {
    +            List<String> topUsers = (List<String>) topologyConf.get(Config.TOPOLOGY_USERS);
    +            for (String logUser : topUsers){
    +                logsUsers.add(logUser);
    +            }
    +        }
    +        data.put(Config.LOGS_USERS, logsUsers.toArray());
    +
    +        File file = ConfigUtils.getLogMetaDataFile(_conf, topologyId, _port);
    +        File parent = file.getParentFile();
    +        if (!Utils.checkFileExists(parent)) {
    +            FileUtils.forceMkdir(file.getParentFile());
    +            _ops.setupStormCodeDir(ConfigUtils.readSupervisorStormConf(_conf, topologyId), file.getParentFile().getCanonicalPath());
    +        }
    +        Yaml yaml = new Yaml();
    +        try (FileWriter writer = new FileWriter(file)) {
    +            yaml.dump(data, writer);
    +        }
    +    }
    +    
    +    /**
    +     * Create symlink from the containers directory/artifacts to the artifacts directory
    +     * @param topologyId the id of the topology this is for
    +     * @throws IOException on any error
    +     */
    +    protected void createArtifactsLink(String topologyId) throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    +        }
    +        String workerDir = ConfigUtils.workerRoot(_conf, _workerId);
    +        String topoDir = ConfigUtils.workerArtifactsRoot(_conf, topologyId);
    +        if (Utils.checkFileExists(workerDir)) {
    +            LOG.debug("Creating symlinks for worker-id: {} topology-id: {} to its port artifacts directory", _workerId, topologyId);
    +            Utils.createSymlink(workerDir, topoDir, "artifacts", String.valueOf(_port));
    +        }
    +    }
    +    
    +    /**
    +     * Create symlinks for each of the blobs from the container's directory to
    +     * corresponding links in the storm dist directory.
    +     * @param topologyId the id of the topology to do this for.
    +     * @throws IOException on any error.
    +     */
    +    protected void createBlobstoreLinks(String topologyId) throws IOException {
    +        String stormRoot = ConfigUtils.supervisorStormDistRoot(_conf, topologyId);
    +        Map<String, Object> stormConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        String workerRoot = ConfigUtils.workerRoot(_conf, _workerId);
    +        
    +        @SuppressWarnings("unchecked")
    +        Map<String, Map<String, Object>> blobstoreMap = (Map<String, Map<String, Object>>) stormConf.get(Config.TOPOLOGY_BLOBSTORE_MAP);
    +        List<String> blobFileNames = new ArrayList<>();
    +        if (blobstoreMap != null) {
    +            for (Map.Entry<String, Map<String, Object>> entry : blobstoreMap.entrySet()) {
    +                String key = entry.getKey();
    +                Map<String, Object> blobInfo = entry.getValue();
    +                String ret = null;
    +                if (blobInfo != null && blobInfo.containsKey("localname")) {
    +                    ret = (String) blobInfo.get("localname");
    +                } else {
    +                    ret = key;
    +                }
    +                blobFileNames.add(ret);
    +            }
    +        }
    +        List<String> resourceFileNames = new ArrayList<>();
    +        resourceFileNames.add(ConfigUtils.RESOURCES_SUBDIR);
    +        resourceFileNames.addAll(blobFileNames);
    +        LOG.info("Creating symlinks for worker-id: {} storm-id: {} for files({}): {}", _workerId, topologyId, resourceFileNames.size(), resourceFileNames);
    +        Utils.createSymlink(workerRoot, stormRoot, ConfigUtils.RESOURCES_SUBDIR);
    +        for (String fileName : blobFileNames) {
    +            Utils.createSymlink(workerRoot, stormRoot, fileName, fileName);
    +        }
    +    }
    +    
    +    /**
    +     * @return all of the pids that are a part of this container.
    +     */
    +    protected Set<Long> getAllPids() throws IOException {
    +        Set<Long> ret = new HashSet<>();
    +        for (String listing: Utils.readDirContents(ConfigUtils.workerPidsRoot(_conf, _workerId))) {
    +            ret.add(Long.valueOf(listing));
    +        }
    +        
    +        if (_resourceIsolationManager != null) {
    +            Set<Long> morePids = _resourceIsolationManager.getRunningPIDs(_workerId);
    +            if (morePids != null) {
    +                ret.addAll(morePids);
    +            }
    +        }
    +        
    +        return ret;
    +    }
    +    
    +    /** 
    +     * @return the user that some operations should be done as.
    +     */
    +    protected String getWorkerUser() {
    +        return ConfigUtils.getWorkerUser(_conf, _workerId);
    +    }
    +    
    +    /**
    +     * Clean up the container partly preparing for restart.
    +     * By default delete all of the temp directories we are going
    +     * to get a new worker_id anyways.
    +     * @throws IOException on any error
    +     */
    +    public void cleanUpForRestart() throws IOException {
    +        // and another API to cleanup with everything is dead
    --- End diff --
    
    TODO?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76875777
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/LocalDownloadedResource.java ---
    @@ -0,0 +1,107 @@
    +/**
    + * 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.localizer;
    +
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class LocalDownloadedResource {
    +    private static final Logger LOG = LoggerFactory.getLogger(LocalDownloadedResource.class);
    +    private static class NoCancelFuture<T> implements Future<T> {
    +        private final Future<T> _wrapped;
    +        
    +        public NoCancelFuture(Future<T> wrapped) {
    +            _wrapped = wrapped;
    +        }
    +        
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            //cancel not currently supported
    --- End diff --
    
    Cancel is difficult to support, but not impossible.  The only value it offers right now would be to possibly stop downloading a jar/blob early if an assignment changed in the middle of the download.  This is rare enough that I didn't feel it was worth the extra effort to do it at this 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76245840
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Supervisor.java ---
    @@ -17,135 +17,541 @@
      */
     package org.apache.storm.daemon.supervisor;
     
    +import java.io.File;
    +import java.io.IOException;
    +import java.net.UnknownHostException;
    +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.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
     import org.apache.commons.io.FileUtils;
     import org.apache.storm.Config;
     import org.apache.storm.StormTimer;
    -import org.apache.storm.daemon.supervisor.timer.RunProfilerActions;
    +import org.apache.storm.cluster.ClusterStateContext;
    +import org.apache.storm.cluster.ClusterUtils;
    +import org.apache.storm.cluster.DaemonType;
    +import org.apache.storm.cluster.IStateStorage;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.daemon.DaemonCommon;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
     import org.apache.storm.daemon.supervisor.timer.SupervisorHealthCheck;
     import org.apache.storm.daemon.supervisor.timer.SupervisorHeartbeat;
     import org.apache.storm.daemon.supervisor.timer.UpdateBlobs;
    +import org.apache.storm.event.EventManager;
     import org.apache.storm.event.EventManagerImp;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.AsyncLocalizer;
    +import org.apache.storm.localizer.ILocalizer;
     import org.apache.storm.localizer.Localizer;
     import org.apache.storm.messaging.IContext;
     import org.apache.storm.metric.StormMetricsRegistry;
     import org.apache.storm.scheduler.ISupervisor;
     import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
     import org.apache.storm.utils.Utils;
     import org.apache.storm.utils.VersionInfo;
    +import org.apache.zookeeper.data.ACL;
    +import org.eclipse.jetty.util.ConcurrentHashSet;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -import java.io.File;
    -import java.io.InterruptedIOException;
    -import java.util.Collection;
    -import java.util.Map;
    -import java.util.Set;
    -import java.util.concurrent.Callable;
    +public class Supervisor implements DaemonCommon, AutoCloseable {
    +    
    +    public class ReadStateThread implements Runnable, AutoCloseable {
    --- End diff --
    
    That sounds reasonable.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78975015
  
    --- 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);
    +        
    +        //Ignore changes to scheduling while downloading the topology blobs
    +        // We don't support canceling the download through the future yet,
    +        // so to keep everything in sync, just wait
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization, staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        // We don't support canceling the download through the future yet,
    +        // so to keep everything in sync, just wait
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization, staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        //The child processes typically exit in < 1 sec.  If 2 mins later they are still around something is wrong
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for WAITING_FOR_WORKER_START state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleWaitingForWorkerStart(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb != null) {
    +            long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +            if (hbAgeMs <= staticState.hbTimeoutMs) {
    +                return dynamicState.withState(MachineState.RUNNING);
    +            }
    +        }
    +        
    +        if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            //We were rescheduled while waiting for the worker to come up
    +            LOG.warn("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAssignment, dynamicState.newAssignment);
    +            return Slot.killContainerForChangedAssignment(dynamicState, staticState);
    --- End diff --
    
    Nitpick: Isn't the Slot. part unnecessary?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76684178
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ContainerLauncher.java ---
    @@ -0,0 +1,99 @@
    +/**
    + * 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.Map;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Launches containers
    + */
    +public abstract class ContainerLauncher {
    +    private static final Logger LOG = LoggerFactory.getLogger(ContainerLauncher.class);
    +    
    +    /**
    +     * Factory to create the right container launcher 
    +     * for the config and the environment.
    +     * @param conf the config
    +     * @param supervisorId the ID of the supervisor
    +     * @param sharedContext Used in local mode to let workers talk together without netty
    +     * @return the proper container launcher
    +     * @throws IOException on any error
    +     */
    +    public static ContainerLauncher mk(Map<String, Object> conf, String supervisorId, IContext sharedContext) throws IOException {
    +        if (ConfigUtils.isLocalMode(conf)) {
    +            return new LocalContainerLauncher(conf, supervisorId, sharedContext);
    +        }
    +        
    +        ResourceIsolationInterface resourceIsolationManager = null;
    +        if (Utils.getBoolean(conf.get(Config.STORM_RESOURCE_ISOLATION_PLUGIN_ENABLE), false)) {
    +            resourceIsolationManager = Utils.newInstance((String) conf.get(Config.STORM_RESOURCE_ISOLATION_PLUGIN));
    --- End diff --
    
    done


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77421324
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    +                            if (actions == null) {
    +                                actions = new HashSet<>();
    +                                filtered.put(port.intValue(), actions);
    +                            }
    +                            actions.add(new TopoProfileAction(topoId, req));
    +                        }
    +                    }
    +                }
    +            }
    +            
    +            for (Integer port: allPorts) {
    +                Slot slot = slots.get(port);
    +                if (slot == null) {
    +                    slot = mkSlot(port);
    +                    slots.put(port, slot);
    +                }
    +                slot.setNewAssignment(allAssignments.get(port));
    +                slot.addProfilerActions(filtered.get(port));
    +            }
    +            
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +    }
    +    
    +    protected Map<String, VersionedData<Assignment>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +            Map<String, VersionedData<Assignment>> localAssignmentVersion, Runnable callback) throws Exception {
    +        Map<String, VersionedData<Assignment>> updateAssignmentVersion = new HashMap<>();
    +        for (String topoId : topoIds) {
    +            Integer recordedVersion = -1;
    +            Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +            VersionedData<Assignment> locAssignment = localAssignmentVersion.get(topoId);
    +            if (locAssignment != null) {
    +                recordedVersion = locAssignment.getVersion();
    +            }
    +            if (version == null) {
    +                // ignore
    +            } else if (version == recordedVersion) {
    +                updateAssignmentVersion.put(topoId, locAssignment);
    +            } else {
    +                VersionedData<Assignment> assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                updateAssignmentVersion.put(topoId, assignmentVersion);
    +            }
    +        }
    +        return updateAssignmentVersion;
    +    }
    +    
    +    protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +        Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +        for (String stormId : stormIds) {
    +            List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +            ret.put(stormId, profileRequests);
    +        }
    +        return ret;
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readAssignments(Map<String, VersionedData<Assignment>> assignmentsSnapshot,
    +            String assignmentId, AtomicInteger retries) {
    +        try {
    +            Map<Integer, LocalAssignment> portLA = new HashMap<Integer, LocalAssignment>();
    +            for (Map.Entry<String, VersionedData<Assignment>> assignEntry : assignmentsSnapshot.entrySet()) {
    +                String topoId = assignEntry.getKey();
    +                Assignment assignment = assignEntry.getValue().getData();
    +
    +                Map<Integer, LocalAssignment> portTasks = readMyExecutors(topoId, assignmentId, assignment);
    +
    +                for (Map.Entry<Integer, LocalAssignment> entry : portTasks.entrySet()) {
    +
    +                    Integer port = entry.getKey();
    +
    +                    LocalAssignment la = entry.getValue();
    +
    +                    if (!portLA.containsKey(port)) {
    +                        portLA.put(port, la);
    +                    } else {
    +                        throw new RuntimeException("Should not have multiple topologies assigned to one port");
    +                    }
    +                }
    +            }
    +            retries.set(0);
    +            return portLA;
    +        } catch (RuntimeException e) {
    +            if (retries.get() > 2) {
    +                throw e;
    +            } else {
    +                retries.addAndGet(1);
    +            }
    +            LOG.warn("{} : retrying {} of 3", e.getMessage(), retries.get());
    +            return null;
    +        }
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readMyExecutors(String stormId, String assignmentId, Assignment assignment) {
    +        Map<Integer, LocalAssignment> portTasks = new HashMap<>();
    +        Map<Long, WorkerResources> slotsResources = new HashMap<>();
    +        Map<NodeInfo, WorkerResources> nodeInfoWorkerResourcesMap = assignment.get_worker_resources();
    +        if (nodeInfoWorkerResourcesMap != null) {
    +            for (Map.Entry<NodeInfo, WorkerResources> entry : nodeInfoWorkerResourcesMap.entrySet()) {
    +                if (entry.getKey().get_node().equals(assignmentId)) {
    +                    Set<Long> ports = entry.getKey().get_port();
    +                    for (Long port : ports) {
    +                        slotsResources.put(port, entry.getValue());
    +                    }
    +                }
    +            }
    +        }
    +        Map<List<Long>, NodeInfo> executorNodePort = assignment.get_executor_node_port();
    +        if (executorNodePort != null) {
    +            for (Map.Entry<List<Long>, NodeInfo> entry : executorNodePort.entrySet()) {
    +                if (entry.getValue().get_node().equals(assignmentId)) {
    +                    for (Long port : entry.getValue().get_port()) {
    +                        LocalAssignment localAssignment = portTasks.get(port.intValue());
    +                        if (localAssignment == null) {
    +                            List<ExecutorInfo> executors = new ArrayList<ExecutorInfo>();
    +                            localAssignment = new LocalAssignment(stormId, executors);
    +                            if (slotsResources.containsKey(port)) {
    +                                localAssignment.set_resources(slotsResources.get(port));
    +                            }
    +                            portTasks.put(port.intValue(), localAssignment);
    +                        }
    +                        List<ExecutorInfo> executorInfoList = localAssignment.get_executors();
    +                        executorInfoList.add(new ExecutorInfo(entry.getKey().get(0).intValue(), entry.getKey().get(entry.getKey().size() - 1).intValue()));
    +                    }
    +                }
    +            }
    +        }
    +        return portTasks;
    +    }
    +
    +    public synchronized void shutdownAllWorkers() {
    +        for (Slot slot: slots.values()) {
    +            try {
    +                slot.setNewAssignment(null);
    +            } catch (Exception e) {
    +                LOG.error("Error trying to shutdown workers in {}", slot, e);
    +            }
    +        }
    +
    +        for (Slot slot: slots.values()) {
    +            try {
    +                int count = 0;
    +                while (slot.getMachineState() != MachineState.EMPTY) {
    +                    if (count > 10) {
    +                        LOG.warn("DONE waiting for {} to finish {}", slot, slot.getMachineState());
    +                        break;
    +                    }
    +                    if (Time.isSimulating()) {
    +                        Time.advanceTime(1000);
    +                        Thread.sleep(100);
    --- End diff --
    
    OK good.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76472957
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,437 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = -1;
    +        _assignment = null;
    +        _workerId = workerId;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq"+pid);
    --- End diff --
    
    Missing a space after `eq` here?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77353254
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    +                            if (actions == null) {
    +                                actions = new HashSet<>();
    +                                filtered.put(port.intValue(), actions);
    +                            }
    +                            actions.add(new TopoProfileAction(topoId, req));
    +                        }
    +                    }
    +                }
    +            }
    +            
    +            for (Integer port: allPorts) {
    +                Slot slot = slots.get(port);
    +                if (slot == null) {
    +                    slot = mkSlot(port);
    +                    slots.put(port, slot);
    +                }
    +                slot.setNewAssignment(allAssignments.get(port));
    +                slot.addProfilerActions(filtered.get(port));
    +            }
    +            
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +    }
    +    
    +    protected Map<String, VersionedData<Assignment>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +            Map<String, VersionedData<Assignment>> localAssignmentVersion, Runnable callback) throws Exception {
    +        Map<String, VersionedData<Assignment>> updateAssignmentVersion = new HashMap<>();
    +        for (String topoId : topoIds) {
    +            Integer recordedVersion = -1;
    +            Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +            VersionedData<Assignment> locAssignment = localAssignmentVersion.get(topoId);
    +            if (locAssignment != null) {
    +                recordedVersion = locAssignment.getVersion();
    +            }
    +            if (version == null) {
    +                // ignore
    +            } else if (version == recordedVersion) {
    +                updateAssignmentVersion.put(topoId, locAssignment);
    +            } else {
    +                VersionedData<Assignment> assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                updateAssignmentVersion.put(topoId, assignmentVersion);
    +            }
    +        }
    +        return updateAssignmentVersion;
    +    }
    +    
    +    protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +        Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +        for (String stormId : stormIds) {
    +            List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +            ret.put(stormId, profileRequests);
    +        }
    +        return ret;
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readAssignments(Map<String, VersionedData<Assignment>> assignmentsSnapshot,
    +            String assignmentId, AtomicInteger retries) {
    +        try {
    +            Map<Integer, LocalAssignment> portLA = new HashMap<Integer, LocalAssignment>();
    +            for (Map.Entry<String, VersionedData<Assignment>> assignEntry : assignmentsSnapshot.entrySet()) {
    +                String topoId = assignEntry.getKey();
    +                Assignment assignment = assignEntry.getValue().getData();
    +
    +                Map<Integer, LocalAssignment> portTasks = readMyExecutors(topoId, assignmentId, assignment);
    +
    +                for (Map.Entry<Integer, LocalAssignment> entry : portTasks.entrySet()) {
    +
    +                    Integer port = entry.getKey();
    +
    +                    LocalAssignment la = entry.getValue();
    +
    +                    if (!portLA.containsKey(port)) {
    +                        portLA.put(port, la);
    +                    } else {
    +                        throw new RuntimeException("Should not have multiple topologies assigned to one port");
    +                    }
    +                }
    +            }
    +            retries.set(0);
    +            return portLA;
    +        } catch (RuntimeException e) {
    +            if (retries.get() > 2) {
    +                throw e;
    +            } else {
    +                retries.addAndGet(1);
    +            }
    +            LOG.warn("{} : retrying {} of 3", e.getMessage(), retries.get());
    +            return null;
    +        }
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readMyExecutors(String stormId, String assignmentId, Assignment assignment) {
    +        Map<Integer, LocalAssignment> portTasks = new HashMap<>();
    +        Map<Long, WorkerResources> slotsResources = new HashMap<>();
    +        Map<NodeInfo, WorkerResources> nodeInfoWorkerResourcesMap = assignment.get_worker_resources();
    +        if (nodeInfoWorkerResourcesMap != null) {
    +            for (Map.Entry<NodeInfo, WorkerResources> entry : nodeInfoWorkerResourcesMap.entrySet()) {
    +                if (entry.getKey().get_node().equals(assignmentId)) {
    +                    Set<Long> ports = entry.getKey().get_port();
    +                    for (Long port : ports) {
    +                        slotsResources.put(port, entry.getValue());
    +                    }
    +                }
    +            }
    +        }
    +        Map<List<Long>, NodeInfo> executorNodePort = assignment.get_executor_node_port();
    +        if (executorNodePort != null) {
    +            for (Map.Entry<List<Long>, NodeInfo> entry : executorNodePort.entrySet()) {
    +                if (entry.getValue().get_node().equals(assignmentId)) {
    +                    for (Long port : entry.getValue().get_port()) {
    +                        LocalAssignment localAssignment = portTasks.get(port.intValue());
    +                        if (localAssignment == null) {
    +                            List<ExecutorInfo> executors = new ArrayList<ExecutorInfo>();
    +                            localAssignment = new LocalAssignment(stormId, executors);
    +                            if (slotsResources.containsKey(port)) {
    +                                localAssignment.set_resources(slotsResources.get(port));
    +                            }
    +                            portTasks.put(port.intValue(), localAssignment);
    +                        }
    +                        List<ExecutorInfo> executorInfoList = localAssignment.get_executors();
    +                        executorInfoList.add(new ExecutorInfo(entry.getKey().get(0).intValue(), entry.getKey().get(entry.getKey().size() - 1).intValue()));
    +                    }
    +                }
    +            }
    +        }
    +        return portTasks;
    +    }
    +
    +    public synchronized void shutdownAllWorkers() {
    +        for (Slot slot: slots.values()) {
    +            try {
    +                slot.setNewAssignment(null);
    +            } catch (Exception e) {
    +                LOG.error("Error trying to shutdown workers in {}", slot, e);
    +            }
    +        }
    +
    +        for (Slot slot: slots.values()) {
    +            try {
    +                int count = 0;
    +                while (slot.getMachineState() != MachineState.EMPTY) {
    +                    if (count > 10) {
    +                        LOG.warn("DONE waiting for {} to finish {}", slot, slot.getMachineState());
    +                        break;
    +                    }
    +                    if (Time.isSimulating()) {
    +                        Time.advanceTime(1000);
    +                        Thread.sleep(100);
    --- End diff --
    
    Should we advance time by the same amount as the sleep 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77725278
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Supervisor.java ---
    @@ -17,135 +17,231 @@
      */
     package org.apache.storm.daemon.supervisor;
     
    +import java.io.File;
    +import java.io.IOException;
    +import java.net.UnknownHostException;
    +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 java.util.concurrent.Callable;
    +import java.util.concurrent.atomic.AtomicReference;
    +
     import org.apache.commons.io.FileUtils;
     import org.apache.storm.Config;
     import org.apache.storm.StormTimer;
    -import org.apache.storm.daemon.supervisor.timer.RunProfilerActions;
    +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.supervisor.timer.SupervisorHealthCheck;
     import org.apache.storm.daemon.supervisor.timer.SupervisorHeartbeat;
     import org.apache.storm.daemon.supervisor.timer.UpdateBlobs;
    +import org.apache.storm.event.EventManager;
     import org.apache.storm.event.EventManagerImp;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.localizer.AsyncLocalizer;
    +import org.apache.storm.localizer.ILocalizer;
     import org.apache.storm.localizer.Localizer;
     import org.apache.storm.messaging.IContext;
     import org.apache.storm.metric.StormMetricsRegistry;
     import org.apache.storm.scheduler.ISupervisor;
     import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
     import org.apache.storm.utils.Utils;
     import org.apache.storm.utils.VersionInfo;
    +import org.apache.zookeeper.data.ACL;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -import java.io.File;
    -import java.io.InterruptedIOException;
    -import java.util.Collection;
    -import java.util.Map;
    -import java.util.Set;
    -import java.util.concurrent.Callable;
    -
    -public class Supervisor {
    +public class Supervisor implements DaemonCommon, AutoCloseable {
         private static final Logger LOG = LoggerFactory.getLogger(Supervisor.class);
    +    private final Map<String, Object> conf;
    +    private final IContext sharedContext;
    +    private volatile boolean active;
    +    private final ISupervisor iSupervisor;
    +    private final Utils.UptimeComputer upTime;
    +    private final String stormVersion;
    +    private final IStormClusterState stormClusterState;
    +    private final LocalState localState;
    +    private final String supervisorId;
    +    private final String assignmentId;
    +    private final String hostName;
    +    // used for reporting used ports when heartbeating
    +    private final AtomicReference<Map<Long, LocalAssignment>> currAssignment;
    --- End diff --
    
    currAssignment is never set.  Only the supervisor hb to nimbus uses this, and nimbus is OK with having used nodes show up as free, so it really only impacts the UI, showing slots as both free and assigned at the same time.  Will fix it.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by ptgoetz <gi...@git.apache.org>.
Github user ptgoetz commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    I have a few style-related nits, but I've refrained from pointing them out because the style(s) in the codebase are all over the place, and the style is inherited in some places. If we want to get nit picky over style, we should really have a style guide in place. That's a JIRA for another day...
    
    Overall I think it looks great and I am +1 for merging. Thanks @revans2 for the great work, and everyone else for the thorough review.
    
    In terms of back porting this to other branches, I'm okay with that. While the impetus behind this work was the Clojure to Java migration, I think the improvements made here warrant back-porting to earlier version branches. It also doesn't really affect user-facing APIs, so that's not a concern.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77415373
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,484 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.io.Reader;
    +import java.io.Writer;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected final Map<String, Object> _topoConf;
    +    protected String _workerId;
    +    protected final String _topologyId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    //Exposed for testing
    +    protected Container(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        assert((assignment == null && port <= 0) ||
    +                (assignment != null && port > 0));
    +        assert(conf != null);
    +        assert(ops != null);
    +        assert(supervisorId != null);
    +        
    +        _port = port;
    +        _ops = ops;
    +        _assignment = assignment;
    +        if (assignment != null) {
    +            _topologyId = assignment.get_topology_id();
    +        } else {
    +            _topologyId = null;
    +        }
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        if (topoConf == null) {
    +            _topoConf = readTopoConf();
    +        } else {
    +            _topoConf = topoConf;
    +        }
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return this.getClass().getSimpleName() + " topo:" + _topologyId + " worker:" + _workerId;
    +    }
    +    
    +    protected Map<String, Object> readTopoConf() throws IOException {
    +        assert(_topologyId != null);
    +        return ConfigUtils.readSupervisorStormConf(_conf, _topologyId);
    +    }
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), port, assignment, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     * @throws IOException on any error
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), -1, null, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.trace("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq" + pid);
    --- End diff --
    
    great catch


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @d2r @abellina @harshach @knusbaum 
    
    I believe that I am done and that the code is ready to merge.  I have addressed all of the review comments to date.  I have finished with updating the unit tests.  I am going to start doing more extensive testing of CGROUP and run as user support, but because we are not too close to a 2.x release I think we can address any issues I find in follow up JIRA.
    
    Please take another look and let me know.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @HeartSaVioR take your time.  I want to be sure that I have plenty of eyeballs looking at this.  We are doing this mostly because we started to run into a lot of race conditions in the supervisor.  We would try to fix one, and another would be exposed.  This makes it so that interacting with the container is only ever done on a single thread so we can reason about the state of the container and avoid those races.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76476005
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,437 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = -1;
    +        _assignment = null;
    +        _workerId = workerId;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq"+pid);
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            if (in.readLine() != null) {
    +                ret = true;
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    private boolean isPosixProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("ps", "-o", "user", "-p", String.valueOf(pid));
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            String first = in.readLine();
    +            assert("USER".equals(first));
    +            String processUser;
    +            while ((processUser = in.readLine()) != null) {
    +                if (user.equals(processUser)) {
    +                    ret = true;
    +                    break;
    +                } else {
    +                    LOG.info("Found {} running as {}, but expected it to be {}", pid, processUser, user);
    +                }
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    @Override
    +    public boolean areAllProcessesDead() throws IOException {
    +        Set<Long> pids = getAllPids();
    +        String user = getWorkerUser();
    +        
    +        boolean allDead = true;
    +        for (Long pid: pids) {
    +            if (!isProcessAlive(pid, user)) {
    +                LOG.warn("{}: PID {} is dead", _workerId, pid);
    +            } else {
    +                allDead = false;
    +                break;
    +            }
    +        }
    +        return allDead;
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +    }
    +
    +    /**
    +     * Setup the container to run.  By default this creates the needed directories/links in the
    +     * local file system
    +     * PREREQUISITE: All needed blobs and topology, jars/configs have been downloaded and
    +     * placed in the appropriate locations
    +     * @throws IOException on any error
    +     */
    +    protected void setup() throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    +        }
    +        final String topologyId = _assignment.get_topology_id();
    +        if (!SupervisorUtils.doRequiredTopoFilesExist(_conf, topologyId)) {
    +            LOG.info("Missing topology storm code, so can't launch  worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                    _supervisorId, _port, _workerId);
    +            throw new IllegalStateException("Not all needed files are here!!!!");
    +        }
    +        String pidsPath = ConfigUtils.workerPidsRoot(_conf, _workerId);
    +        String hbPath = ConfigUtils.workerHeartbeatsRoot(_conf, _workerId);
    +    
    +        FileUtils.forceMkdir(new File(pidsPath));
    +        FileUtils.forceMkdir(new File(ConfigUtils.workerTmpRoot(_conf, _workerId)));
    +        FileUtils.forceMkdir(new File(hbPath));
    +    
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        String user = (String) topologyConf.get(Config.TOPOLOGY_SUBMITTER_USER);
    +        writeLogMetadata(topologyConf, user, topologyId);
    +        ConfigUtils.setWorkerUserWSE(_conf, _workerId, user);
    +        createArtifactsLink(topologyId);
    +    
    +        createBlobstoreLinks(topologyId);
    +    }
    +    
    +    /**
    +     * Write out the file used by the log viewer to allow/reject log access
    +     * @param topologyConf the config for the topology
    +     * @param user the user this is going to run as
    +     * @param topologyId the id of the topology
    +     * @throws IOException on any error
    +     */
    +    @SuppressWarnings("unchecked")
    +    protected void writeLogMetadata(Map<String, Object> topologyConf, String user, String topologyId) throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    +        }
    +        Map<String, Object> data = new HashMap<>();
    +        data.put(Config.TOPOLOGY_SUBMITTER_USER, user);
    +        data.put("worker-id", _workerId);
    +
    +        Set<String> logsGroups = new HashSet<>();
    +        //for supervisor-test
    --- End diff --
    
    Why is this needed only for the test?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77442649
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    --- End diff --
    
    Changing it to `this.assignmentVersions = new AtomicReference<>(new HashMap<>());` compiles fine for me on Java 8.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77411245
  
    --- Diff: storm-core/src/jvm/org/apache/storm/utils/ConfigUtils.java ---
    @@ -353,25 +350,21 @@ public LocalState nimbusTopoHistoryStateImpl(Map conf) throws IOException {
         }
     
         // we use this "weird" wrapper pattern temporarily for mocking in clojure test
    -    public static Map readSupervisorStormConf(Map conf, String stormId) throws IOException {
    +    public static Map<String, Object> readSupervisorStormConf(Map<String, Object> conf, String stormId) throws IOException {
             return _instance.readSupervisorStormConfImpl(conf, stormId);
         }
     
    -    public Map readSupervisorStormConfImpl(Map conf, String stormId) throws IOException {
    +    public Map<String, Object> readSupervisorStormConfImpl(Map<String, Object> conf, String stormId) throws IOException {
             String stormRoot = supervisorStormDistRoot(conf, stormId);
             String confPath = supervisorStormConfPath(stormRoot);
             return readSupervisorStormConfGivenPath(conf, confPath);
         }
     
         // we use this "weird" wrapper pattern temporarily for mocking in clojure test
    -    public static StormTopology readSupervisorTopology(Map conf, String stormId) throws IOException {
    -        return _instance.readSupervisorTopologyImpl(conf, stormId);
    -    }
    -
    -    public StormTopology readSupervisorTopologyImpl(Map conf, String stormId) throws IOException {
    +    public static StormTopology readSupervisorTopology(Map conf, String stormId, AdvancedFSOps ops) throws IOException {
    --- End diff --
    
    I put in the AdvancedFSOps to handle the mocking, but now I am not so sure on that.  It is too low level for a lot of what we want to do, so I think I will put it back in, and just do the mocking at the higher level.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77416200
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    --- End diff --
    
    Ya that comes from thrift, and I really don't know why it is a Long there.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    I was able to review most of the code.  I skimmed the tests and didn't see anything strange. 
    
    @revans2 knows that I will not be able to log on again for awhile.  I want to leave a note for others that, as my comments were either minor or were something we discussed, I don't want to hold up a merge due to my unresolved review comments should this change otherwise be ready to merge before I am back.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76796976
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,313 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.Writer;
    +import java.nio.file.FileSystems;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    +    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
    +    
    +    /**
    +     * Factory to create a new AdvancedFSOps
    +     * @param conf the configuration of the process
    +     * @return the appropriate instance of the class for this config and environment.
    +     */
    +    public static AdvancedFSOps make(Map<String, Object> conf) {
    +        if (Utils.isOnWindows()) {
    +            return new AdvancedWindowsFSOps(conf);
    +        }
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new AdvancedRunAsUserFSOps(conf);
    +        }
    +        return new AdvancedFSOps();
    +    }
    +    
    +    private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
    +        private final Map<String, Object> _conf;
    +        
    +        public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
    +            if (Utils.isOnWindows()) {
    +                throw new UnsupportedOperationException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +            _conf = conf;
    +        }
    +        
    +        @Override
    +        public void setupBlobPermissions(File path, String user) throws IOException {
    +            String logPrefix = "setup blob permissions for " + path;
    +            SupervisorUtils.processLauncherAndWait(_conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
    +        }
    +        
    +        @Override
    +        public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +            String absolutePath = path.getAbsolutePath();
    +            LOG.info("Deleting path {}", absolutePath);
    +            if (user == null) {
    +                user = Files.getOwner(path.toPath()).getName();
    +            }
    +            List<String> commands = new ArrayList<>();
    +            commands.add("rmr");
    +            commands.add(absolutePath);
    +            SupervisorUtils.processLauncherAndWait(_conf, user, commands, null, logPrefix);
    +            if (Utils.checkFileExists(absolutePath)) {
    +                throw new RuntimeException(path + " was not deleted.");
    +            }
    +        }
    +        
    +        @Override
    +        public void setupStormCodeDir(Map<String, Object> topologyConf, File path) throws IOException {
    +            SupervisorUtils.setupStormCodeDir(_conf, topologyConf, path.getCanonicalPath());
    +        }
    +    }
    +    
    +    /**
    +     * Operations that need to override the default ones when running on Windows
    +     *
    +     */
    +    private static class AdvancedWindowsFSOps extends AdvancedFSOps {
    +
    +        public AdvancedWindowsFSOps(Map<String, Object> conf) {
    +            if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +        }
    +        
    +        @Override
    +        public void restrictDirectoryPermissions(String dir) throws IOException {
    +            //NOOP, if windows gets support for run as user we will need to find a way to suppor this
    +        }
    +        
    +        @Override
    +        public void moveDriectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // This is not atomic but it does work
    +            FileUtils.moveDirectory(fromDir, toDir);
    +        }
    +        
    +        @Override
    +        public boolean supportsAtomicDirectoryMove() {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // FileUtils.moveDirectory is not atomic
    +            return false;
    +        }
    +    }
    +    
    +    
    +    protected AdvancedFSOps() {
    +        //NOOP, but restricted permissions
    +    }
    +
    +    /**
    +     * Set directory permissions to (OWNER)RWX (GROUP)R-X (OTHER)---
    +     * On some systems that do not support this, it may become a noop
    +     * @param dir the directory to change permissions on
    +     * @throws IOException on any error
    +     */
    +    public void restrictDirectoryPermissions(String dir) throws IOException {
    +        Set<PosixFilePermission> perms = new HashSet<>(
    +                Arrays.asList(PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE,
    +                        PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ,
    +                        PosixFilePermission.GROUP_EXECUTE));
    +        Files.setPosixFilePermissions(FileSystems.getDefault().getPath(dir), perms);
    +    }
    +
    +    /**
    +     * Move fromDir to toDir, and try to make it an atomic move if possible
    +     * @param fromDir what to move
    +     * @param toDir where to move it from
    +     * @throws IOException on any error
    +     */
    +    public void moveDriectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +        FileUtils.forceMkdir(toDir);
    +        Files.move(fromDir.toPath(), toDir.toPath(), StandardCopyOption.ATOMIC_MOVE);
    +    }
    +    
    +    /**
    +     * @return true if an atomic directory move works, else false.
    +     */
    +    public boolean supportsAtomicDirectoryMove() {
    +        return true;
    +    }
    +    
    +    /**
    +     * Setup permissions properly for an internal blob store path
    +     * @param path the path to set the permissions on
    +     * @param user the user to change the permissions for
    +     * @throws IOException on any error
    +     */
    +    public void setupBlobPermissions(File path, String user) throws IOException {
    +        //Normally this is a NOOP
    +    }
    +
    +    /**
    +     * Delete a file or a directory and all of the children. If it exists.
    +     * @param path what to delete
    +     * @param user who to delete it as if doing it as someone else is supported
    +     * @param logPrefix if an external process needs to be launched to delete 
    +     * the object what prefix to include in the logs
    +     * @throws IOException on any error.
    +     */
    +    public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +        LOG.info("Deleting path {}", path);
    +        Path p = path.toPath();
    +        if (Files.exists(p)) {
    +            try {
    +                FileUtils.forceDelete(path);
    +            } catch (FileNotFoundException ignored) {}
    +        }
    +    }
    +    
    +    /**
    +     * Delete a file or a directory and all of the children. If it exists.
    +     * @param path what to delete
    +     * @throws IOException on any error.
    +     */
    +    public void deleteIfExists(File path) throws IOException {
    +        LOG.info("Deleting path {}", path);
    +        Path p = path.toPath();
    +        if (Files.exists(p)) {
    +            try {
    +                FileUtils.forceDelete(path);
    +            } catch (FileNotFoundException ignored) {}
    +        }
    +    }
    +
    +    /**
    +     * Setup the permissions for the storm code dir
    +     * @param topologyConf the config of the Topology
    +     * @param path the directory to set the permissions on
    +     * @throws IOException on any error
    +     */
    +    public void setupStormCodeDir(Map<String, Object> topologyConf, File path) throws IOException {
    +        //By default this is a NOOP
    +    }
    +
    +    /**
    +     * Sanity check if everything the topology needs is there for it to run.
    +     * @param conf the config of the supervisor
    +     * @param topologyId the ID of the topology
    +     * @return true if everything is there, else false
    +     * @throws IOException on any error
    +     */
    +    public boolean doRequiredTopoFilesExist(Map<String, Object> conf, String topologyId) throws IOException {
    +        return SupervisorUtils.doRequiredTopoFilesExist(conf, topologyId);
    +    }
    +    
    +    /**
    +     * Makes a directory, including any necessary but nonexistent parent
    +     * directories. 
    +     *
    +     * @param path the directory to create
    +     * @throws IOException on any error
    +     */
    +    public void forceMkdir(File path) throws IOException {
    +        FileUtils.forceMkdir(path);
    +    }
    +    
    +    /**
    +     * Check if a file exists or not
    +     * @param path the path the check
    +     * @return true if it exists else false
    +     * @throws IOException on any error.
    +     */
    +    public boolean fileExists(File path) throws IOException {
    +        return path.exists();
    +    }
    +
    +    /**
    +     * Get a writer for the given location 
    +     * @param file the file to write to
    +     * @return the Writer to use.
    +     * @throws IOException on any error
    +     */
    +    public Writer getWriter(File file) throws IOException {
    +        return new FileWriter(file);
    +    }
    +    
    +    /**
    +     * Dump a string to a file
    +     * @param location where to write to
    +     * @param data the data to write
    +     * @throws IOException on any error
    +     */
    +    public void dump(File location, String data) throws IOException {
    +        File parent = location.getParentFile();
    +        if (!parent.exists()) {
    +            forceMkdir(parent);
    +        }
    +        try (Writer w = getWriter(location)) {
    +            w.write(data);
    +        }
    +    }
    +    
    +    /**
    +     * Read the contents of a file into a String
    +     * @param location the file to read
    +     * @return the contents of the file
    +     * @throws IOException on any error
    +     */
    +    public String slurpString(File location) throws IOException {
    +        return FileUtils.readFileToString(location, "UTF-8");
    +    }
    +    
    +    public byte[] slurp(File location) throws IOException {
    --- End diff --
    
    nit, missing a function header.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77358057
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    +                            if (actions == null) {
    +                                actions = new HashSet<>();
    +                                filtered.put(port.intValue(), actions);
    +                            }
    +                            actions.add(new TopoProfileAction(topoId, req));
    +                        }
    +                    }
    +                }
    +            }
    +            
    +            for (Integer port: allPorts) {
    +                Slot slot = slots.get(port);
    +                if (slot == null) {
    +                    slot = mkSlot(port);
    +                    slots.put(port, slot);
    +                }
    +                slot.setNewAssignment(allAssignments.get(port));
    +                slot.addProfilerActions(filtered.get(port));
    +            }
    +            
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +    }
    +    
    +    protected Map<String, VersionedData<Assignment>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +            Map<String, VersionedData<Assignment>> localAssignmentVersion, Runnable callback) throws Exception {
    +        Map<String, VersionedData<Assignment>> updateAssignmentVersion = new HashMap<>();
    +        for (String topoId : topoIds) {
    +            Integer recordedVersion = -1;
    +            Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +            VersionedData<Assignment> locAssignment = localAssignmentVersion.get(topoId);
    +            if (locAssignment != null) {
    +                recordedVersion = locAssignment.getVersion();
    +            }
    +            if (version == null) {
    +                // ignore
    +            } else if (version == recordedVersion) {
    +                updateAssignmentVersion.put(topoId, locAssignment);
    +            } else {
    +                VersionedData<Assignment> assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                updateAssignmentVersion.put(topoId, assignmentVersion);
    +            }
    +        }
    +        return updateAssignmentVersion;
    +    }
    +    
    +    protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +        Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +        for (String stormId : stormIds) {
    +            List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +            ret.put(stormId, profileRequests);
    +        }
    +        return ret;
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readAssignments(Map<String, VersionedData<Assignment>> assignmentsSnapshot,
    +            String assignmentId, AtomicInteger retries) {
    +        try {
    +            Map<Integer, LocalAssignment> portLA = new HashMap<Integer, LocalAssignment>();
    +            for (Map.Entry<String, VersionedData<Assignment>> assignEntry : assignmentsSnapshot.entrySet()) {
    +                String topoId = assignEntry.getKey();
    +                Assignment assignment = assignEntry.getValue().getData();
    +
    +                Map<Integer, LocalAssignment> portTasks = readMyExecutors(topoId, assignmentId, assignment);
    +
    +                for (Map.Entry<Integer, LocalAssignment> entry : portTasks.entrySet()) {
    +
    +                    Integer port = entry.getKey();
    +
    +                    LocalAssignment la = entry.getValue();
    +
    +                    if (!portLA.containsKey(port)) {
    +                        portLA.put(port, la);
    +                    } else {
    +                        throw new RuntimeException("Should not have multiple topologies assigned to one port");
    +                    }
    +                }
    +            }
    +            retries.set(0);
    +            return portLA;
    +        } catch (RuntimeException e) {
    +            if (retries.get() > 2) {
    +                throw e;
    +            } else {
    +                retries.addAndGet(1);
    +            }
    +            LOG.warn("{} : retrying {} of 3", e.getMessage(), retries.get());
    --- End diff --
    
    minor, but if retries happens to get changed in another thread, we would be logging the incorrect retry count here.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77154990
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,493 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.io.Reader;
    +import java.io.Writer;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected final Map<String, Object> _topoConf;
    +    protected String _workerId;
    +    protected final String _topologyId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    //Exposed for testing
    +    protected Container(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        assert((assignment == null && port <= 0) ||
    +                (assignment != null && port > 0));
    +        assert(conf != null);
    +        assert(ops != null);
    +        assert(supervisorId != null);
    +        
    +        _port = port;
    +        _ops = ops;
    +        _assignment = assignment;
    +        if (assignment != null) {
    +            _topologyId = assignment.get_topology_id();
    +        } else {
    +            _topologyId = null;
    +        }
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        if (topoConf == null) {
    +            _topoConf = readTopoConf();
    +        } else {
    +            _topoConf = topoConf;
    +        }
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return this.getClass().getSimpleName() + " topo:" + _topologyId + " worker:" + _workerId;
    +    }
    +    
    +    protected Map<String, Object> readTopoConf() throws IOException {
    +        assert(_topologyId != null);
    +        return ConfigUtils.readSupervisorStormConf(_conf, _topologyId);
    +    }
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), port, assignment, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     * @throws IOException on any error
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), -1, null, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.trace("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    --- End diff --
    
    Sorry, was a bit too fast there. tasklist also outputs user name in domain\username format if passed the /v flag.
    `C:\Users\Stig>tasklist /nh /fi "pid eq 10300" /v`
    outputs
    `bash.exe                     10300 Console                   21        888 K Unknown         STIG-LAPTOP\Stig                                        0:00:00 N/A` 
    
    The columns are "Image Name, PID, Session Name, Session#, Mem Usage, Status, User Name, CPU Time, Window Title"


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77253867
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,629 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +
    +    //For testing purposes
    +    public BasicContainer(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager, LocalState localState,
    +            String profileCmd) throws IOException {
    +        super(ops, port, assignment, conf, topoConf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +        _profileCmd = profileCmd;
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState localState, ResourceIsolationInterface resourceIsolationManager, boolean recover)
    +            throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException(
    +                    "Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            _localState.setApprovedWorkers(workersToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        setup();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = _ops.slurpString(new File(str)).trim();
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        if (command.size() > 0) {
    +            return runProfilingCommand(command, env, logPrefix, targetFile);
    +        }
    +        LOG.warn("PROFILING REQUEST NOT SUPPORTED {} IGNORED...", request);
    +        return true;
    +    }
    +
    +    /**
    +     * Get the command to run when doing profiling
    +     * @param action the profiling action to perform
    +     * @param stop if this is meant to stop the profiling or start it
    +     * @param workerPid the PID of the process to profile
    +     * @param targetDir the current working directory of the worker process
    +     * @return the command to run for profiling.
    +     */
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        switch(action) {
    +            case JMAP_DUMP:
    +                return jmapDumpCmd(workerPid, targetDir);
    +            case JSTACK_DUMP:
    +                return jstackDumpCmd(workerPid, targetDir);
    +            case JPROFILE_DUMP:
    +                return jprofileDump(workerPid, targetDir);
    +            case JVM_RESTART:
    +                return jprofileJvmRestart(workerPid);
    +            case JPROFILE_STOP:
    +                if (stop) {
    +                    return jprofileStop(workerPid, targetDir);
    +                }
    +                return jprofileStart(workerPid);
    +            default:
    +                return Lists.newArrayList();
    +        }
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +
    +    /**
    +     * Compute the java.library.path that should be used for the worker.
    +     * This helps it to load JNI libraries that are packaged in the uber jar.
    +     * @param stormRoot the root directory of the worker process
    +     * @param conf the config for the supervisor.
    +     * @return the java.library.path/LD_LIBRARY_PATH to use so native libraries load correctly.
    +     */
    +    protected String jlp(String stormRoot, Map<String, Object> conf) {
    +        String resourceRoot = stormRoot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +        String os = System.getProperty("os.name").replaceAll("\\s+", "_");
    +        String arch = System.getProperty("os.arch");
    +        String archResourceRoot = resourceRoot + Utils.FILE_PATH_SEPARATOR + os + "-" + arch;
    +        String ret = CPJ.join(archResourceRoot, resourceRoot,
    +                conf.get(Config.JAVA_LIBRARY_PATH));
    +        return ret;
    +    }
    +
    +    /**
    +     * Returns a collection of jar file names found under the given directory.
    +     * @param dir the directory to search
    +     * @return the jar file names
    +     */
    +    protected List<String> getFullJars(File dir) {
    +        File[] files = dir.listFiles(jarFilter);
    +
    +        if (files == null) {
    +            return Collections.emptyList();
    +        }
    +
    +        return Arrays.stream(files).map(f -> f.getAbsolutePath())
    +                .collect(Collectors.toList());
    +    }
    +    
    +    protected List<String> frameworkClasspath() {
    +        String stormHome = System.getProperty("storm.home");
    +
    +        File stormLibDir = new File(stormHome, "lib");
    +        String stormConfDir =
    +                System.getenv("STORM_CONF_DIR") != null ?
    +                System.getenv("STORM_CONF_DIR") :
    +                new File(stormHome, "conf").getAbsolutePath();
    +        File stormExtlibDir = new File(stormHome, "extlib");
    +        String extcp = System.getenv("STORM_EXT_CLASSPATH");
    +        List<String> pathElements = new LinkedList<>();
    +        pathElements.addAll(getFullJars(stormLibDir));
    +        pathElements.addAll(getFullJars(stormExtlibDir));
    +        pathElements.add(extcp);
    +        pathElements.add(stormConfDir);
    +
    +        return pathElements;
    +    }
    +    
    +    @SuppressWarnings("unchecked")
    +    private List<String> asStringList(Object o) {
    +        if (o instanceof String) {
    +            return Arrays.asList((String)o);
    +        } else if (o instanceof List) {
    +            return (List<String>)o;
    +        }
    +        return Collections.EMPTY_LIST;
    +    }
    +    
    +    /**
    +     * Compute the classpath for the worker process
    +     * @param stormJar the topology jar
    +     * @param dependencyLocations any dependencies from the topology
    +     * @return the full classpath
    +     */
    +    protected String getWorkerClassPath(String stormJar, List<String> dependencyLocations) {
    +        List<String> workercp = new ArrayList<>();
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH_BEGINNING)));
    +        workercp.addAll(frameworkClasspath());
    +        workercp.add(stormJar);
    +        workercp.addAll(dependencyLocations);
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH)));
    +        return CPJ.join(workercp);
    +    }
    +
    +    private String substituteChildOptsInternal(String string, int memOnheap) {
    +        if (StringUtils.isNotBlank(string)) {
    +            String p = String.valueOf(_port);
    +            string = string.replace("%ID%", p);
    +            string = string.replace("%WORKER-ID%", _workerId);
    +            string = string.replace("%TOPOLOGY-ID%", _topologyId);
    +            string = string.replace("%WORKER-PORT%", p);
    +            if (memOnheap > 0) {
    +                string = string.replace("%HEAP-MEM%", String.valueOf(memOnheap));
    +            }
    +        }
    +        return string;
    +    }
    +    
    +    protected List<String> substituteChildopts(Object value) {
    +        return substituteChildopts(value, -1);
    +    }
    +
    +    protected List<String> substituteChildopts(Object value, int memOnheap) {
    +        List<String> rets = new ArrayList<>();
    +        if (value instanceof String) {
    +            String string = substituteChildOptsInternal((String) value, memOnheap);
    +            if (StringUtils.isNotBlank(string)) {
    +                String[] strings = string.split("\\s+");
    +                rets.addAll(Arrays.asList(strings));
    +            }
    +        } else if (value instanceof List) {
    +            @SuppressWarnings("unchecked")
    +            List<String> objects = (List<String>) value;
    +            for (String object : objects) {
    +                String str = substituteChildOptsInternal(object, memOnheap);
    +                if (StringUtils.isNotBlank(str)) {
    +                    rets.add(str);
    +                }
    +            }
    +        }
    +        return rets;
    +    }
    +
    +    /**
    +     * Launch the worker process (non-blocking)
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param processExitcallback
    +     *            a callback for when the process exits
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     */
    +    protected void launchWorkerProcess(List<String> command, Map<String, String> env, String logPrefix,
    +            ExitCodeCallback processExitCallback, File targetDir) throws IOException {
    +        SupervisorUtils.launchProcess(command, env, logPrefix, processExitCallback, targetDir);
    +    }
    +
    +    private String getWorkerLoggingConfigFile(String stormHome) {
    +        String log4jConfigurationDir = (String) (_conf.get(Config.STORM_LOG4J2_CONF_DIR));
    +
    +        if (StringUtils.isNotBlank(log4jConfigurationDir)) {
    +            if (!Utils.isAbsolutePath(log4jConfigurationDir)) {
    +                log4jConfigurationDir = stormHome + Utils.FILE_PATH_SEPARATOR + log4jConfigurationDir;
    +            }
    +        } else {
    +            log4jConfigurationDir = stormHome + Utils.FILE_PATH_SEPARATOR + "log4j2";
    +        }
    +        
    +
    +        if (Utils.IS_ON_WINDOWS && !log4jConfigurationDir.startsWith("file:")) {
    +            log4jConfigurationDir = "file:///" + log4jConfigurationDir;
    +        }
    +        return log4jConfigurationDir + Utils.FILE_PATH_SEPARATOR + "worker.xml";
    +    }
    +    
    +    /**
    +     * Get parameters for the class path of the worker process.  Also used by the
    +     * log Writer
    +     * @param stormRoot the root dist dir for the topology
    +     * @return the classpath for the topology as command line arguments.
    +     * @throws IOException on any error.
    +     */
    +    private List<String> getClassPathParams(final String stormRoot) throws IOException {
    +        final String stormJar = ConfigUtils.supervisorStormJarPath(stormRoot);
    +        final StormTopology stormTopology = ConfigUtils.readSupervisorTopology(_conf, _topologyId, _ops);
    +        final List<String> dependencyLocations = new ArrayList<>();
    +        if (stormTopology.get_dependency_jars() != null) {
    +            for (String dependency : stormTopology.get_dependency_jars()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +
    +        if (stormTopology.get_dependency_artifacts() != null) {
    +            for (String dependency : stormTopology.get_dependency_artifacts()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +        final String workerClassPath = getWorkerClassPath(stormJar, dependencyLocations);
    +        
    +        List<String> classPathParams = new ArrayList<>();
    +        classPathParams.add("-cp");
    +        classPathParams.add(workerClassPath);
    +        return classPathParams;
    +    }
    +    
    +    /**
    +     * Get a set of java properties that are common to both the log writer and the worker processes.
    +     * These are mostly system properties that are used by logging.
    +     * @return a list of command line options
    +     */
    +    private List<String> getCommonParams() {
    +        final String stormHome = ConfigUtils.concatIfNotNull(System.getProperty("storm.home"));
    +        final String workersArtifacts = ConfigUtils.workerArtifactsRoot(_conf);
    +        String stormLogDir = ConfigUtils.getLogDir();
    +        String log4jConfigurationFile = getWorkerLoggingConfigFile(stormHome);
    +        
    +        List<String> commonParams = new ArrayList<>();
    +        commonParams.add("-Dlogfile.name=worker.log");
    +        commonParams.add("-Dstorm.home=" + stormHome);
    +        commonParams.add("-Dworkers.artifacts=" + workersArtifacts);
    +        commonParams.add("-Dstorm.id=" + _topologyId);
    +        commonParams.add("-Dworker.id=" + _workerId);
    +        commonParams.add("-Dworker.port=" + _port);
    +        commonParams.add("-Dstorm.log.dir=" + stormLogDir);
    +        commonParams.add("-Dlog4j.configurationFile=" + log4jConfigurationFile);
    +        commonParams.add("-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector");
    +        return commonParams;
    +    }
    +    
    +    private int getMemOnHeap(WorkerResources resources) {
    +        int memOnheap = 0;
    +        if (resources != null && resources.is_set_mem_on_heap() && 
    +                resources.get_mem_on_heap() > 0) {
    +            memOnheap = (int) Math.ceil(resources.get_mem_on_heap());
    +        } else {
    +            // set the default heap memory size for supervisor-test
    +            memOnheap = Utils.getInt(_topoConf.get(Config.WORKER_HEAP_MEMORY_MB), 768);
    +        }
    +        return memOnheap;
    +    }
    +    
    +    private List<String> getWorkerProfilerChildOpts(int memOnheap) {
    +        List<String> workerProfilerChildopts = new ArrayList<>();
    +        if (Utils.getBoolean(_conf.get(Config.WORKER_PROFILER_ENABLED), false)) {
    +            workerProfilerChildopts = substituteChildopts(_conf.get(Config.WORKER_PROFILER_CHILDOPTS), memOnheap);
    +        }
    +        return workerProfilerChildopts;
    +    }
    +    
    +    /**
    +     * a or b the first one that is not null
    +     * @param a something
    +     * @param b something else
    +     * @return a or b the first one that is not null
    +     */
    +    private <V> V OR(V a, V b) {
    +        return a == null ? b : a;
    +    }
    +    
    +    protected String javaCmd(String cmd) {
    +        String ret = null;
    +        String javaHome = System.getenv().get("JAVA_HOME");
    +        if (StringUtils.isNotBlank(javaHome)) {
    +            ret = javaHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR + cmd;
    +        } else {
    +            ret = cmd;
    +        }
    +        return ret;
    +    }
    +    
    +    /**
    +     * Create the command to launch the worker process
    +     * @param memOnheap the on heap memory for the worker
    +     * @param stormRoot the root dist dir for the topology
    +     * @param jlp java library path for the topology
    +     * @return the command to run
    +     * @throws IOException on any error.
    +     */
    +    private List<String> mkLaunchCommand(final int memOnheap, final String stormRoot,
    +            final String jlp) throws IOException {
    +        final String javaCmd = javaCmd("java");
    +        final String stormOptions = ConfigUtils.concatIfNotNull(System.getProperty("storm.options"));
    +        final String stormConfFile = ConfigUtils.concatIfNotNull(System.getProperty("storm.conf.file"));
    +        final String workerTmpDir = ConfigUtils.workerTmpRoot(_conf, _workerId);
    +        
    +        List<String> classPathParams = getClassPathParams(stormRoot);
    +        List<String> commonParams = getCommonParams();
    +        
    +        List<String> commandList = new ArrayList<>();
    +        //Log Writer Command...
    +        commandList.add(javaCmd);
    +        commandList.addAll(classPathParams);
    +        commandList.addAll(substituteChildopts(_topoConf.get(Config.TOPOLOGY_WORKER_LOGWRITER_CHILDOPTS)));
    +        commandList.addAll(commonParams);
    +        commandList.add("org.apache.storm.LogWriter"); //The LogWriter in turn launches the actual worker.
    +
    +        //Worker Command...
    +        commandList.add(javaCmd);
    +        commandList.add("-server");
    +        commandList.addAll(commonParams);
    +        commandList.addAll(substituteChildopts(_conf.get(Config.WORKER_CHILDOPTS), memOnheap));
    +        commandList.addAll(substituteChildopts(_topoConf.get(Config.TOPOLOGY_WORKER_CHILDOPTS), memOnheap));
    +        commandList.addAll(substituteChildopts(OR(
    +                _topoConf.get(Config.TOPOLOGY_WORKER_GC_CHILDOPTS),
    +                _conf.get(Config.WORKER_GC_CHILDOPTS)), memOnheap));
    +        commandList.addAll(getWorkerProfilerChildOpts(memOnheap));
    +        commandList.add("-Djava.library.path=" + jlp);
    +        commandList.add("-Dstorm.conf.file=" + stormConfFile);
    +        commandList.add("-Dstorm.options=" + stormOptions);
    +        commandList.add("-Djava.io.tmpdir=" + workerTmpDir);
    +        commandList.add("-Dlogging.sensitivity=" + OR((String) _topoConf.get(Config.TOPOLOGY_LOGGING_SENSITIVITY), "S3"));
    +        commandList.addAll(classPathParams);
    +        commandList.add("org.apache.storm.daemon.worker");
    +        commandList.add(_topologyId);
    +        commandList.add(_supervisorId);
    +        commandList.add(String.valueOf(_port));
    +        commandList.add(_workerId);
    +        
    +        return commandList;
    +    }
    +    
    +    @Override
    +    public void launch() throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot launch a container recovered with just a worker id");
    +        }
    +        LOG.info("Launching worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                _supervisorId, _port, _workerId);
    +        String logPrefix = "Worker Process " + _workerId;
    +        ProcessExitCallback processExitCallback = new ProcessExitCallback(logPrefix);
    +        _exitedEarly = false;
    +        
    +        final WorkerResources resources = _assignment.get_resources();
    +        final int memOnheap = getMemOnHeap(resources);
    +        final String stormRoot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +        final String jlp = jlp(stormRoot, _conf);
    +        
    +        List<String> commandList = mkLaunchCommand(memOnheap, stormRoot, jlp);
    +
    +        Map<String, String> topEnvironment = new HashMap<String, String>();
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> environment = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (environment != null) {
    +            topEnvironment.putAll(environment);
    +        }
    +        topEnvironment.put("LD_LIBRARY_PATH", jlp);
    +
    +        // {"cpu" cpu "memory" (+ mem-onheap mem-offheap (int (Math/ceil (conf
    --- End diff --
    
    Never mind I'll remove it.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78973993
  
    --- 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);
    +        
    +        //Ignore changes to scheduling while downloading the topology blobs
    +        // We don't support canceling the download through the future yet,
    +        // so to keep everything in sync, just wait
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization, staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        // We don't support canceling the download through the future yet,
    +        // so to keep everything in sync, just wait
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization, staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    --- End diff --
    
    I know that it gets overwritten immediately after, but if you set the slot state to empty when cleaning, you can get rid of the null check in cleanupCurrentContainer


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76269608
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,494 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +    
    +    private class ProcessExitCallback implements Utils.ExitCodeCallable<Void> {
    --- End diff --
    
    can we move the ExitCodeCallable out of Utils , seems that we can have Processor package .


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    I just pushed in a much of new fixes and addressed all of the outstanding review comments.  I think it is good to go.  Please take another look/test it and let me know.
    
    I have run tests with rebalancing, rolling upgrades, Recovery, deleting directories from distcache, and added directories to distcache.  


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @HeartSaVioR I figured out what happened to make the supervisor crash.  In the transition from RUNNING to KILL to speed things up slot starts localizing resources for the new assignment.  In the transition from KILL to WAITING_FOR_BASIC_LOCALIZATION slot releases the resources for the container that was just killed.  But the reference counting is only for the topology id and port.  If the port and topology id are the same, then we get this.  I need a key that is specific to the assignment/port, not the topology/port.  I'll try to figure out what that should look like.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @srdo Pushed a new version that hopefully addresses your review comments.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    And I saw some places using lambda which can't be applied to 1.x. 
    @revans2 Could you craft a pull request for 1.x branch (might also 1.0.x) too when we're sure it's good to merge?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78230264
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,780 @@
    +/**
    + * 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) {
    --- End diff --
    
    nit, could return: withPendingLocalization(this.pendingLocalization, pendingDownload);


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76270667
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,494 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +    
    +    private class ProcessExitCallback implements Utils.ExitCodeCallable<Void> {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public Void call() throws Exception {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +            return null;
    +        }
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, LocalState localState, 
    +            ResourceIsolationInterface resourceIsolationManager, boolean recover) throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized(localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry: workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port +" "+ assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +        
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +    
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized(_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +        synchronized(_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            _localState.setApprovedWorkers(workersToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +    
    +    /**
    +     * Run the given command for profiling
    +     * @param command the command to run
    +     * @param env the environment to run the command
    +     * @param logPrefix the prefix to include in the logs
    +     * @param targetDir the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException on any error
    +     * @throws InterruptedException if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix, File targetDir) throws IOException, InterruptedException {
    +        Process p = Utils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +    
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String topologyId = _assignment.get_topology_id();
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, topologyId, _port);
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) topologyConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, topologyId, _port);
    +
    +        String workerPid = null;
    +        try (FileReader reader = new FileReader(str); BufferedReader br = new BufferedReader(reader)) {
    +            workerPid = br.readLine().trim();
    +        }
    +        
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        return runProfilingCommand(command, env, logPrefix, targetFile);
    +    }
    +    
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        if (action == ProfileAction.JMAP_DUMP) {
    +            return jmapDumpCmd(workerPid, targetDir);
    +        } else if (action == ProfileAction.JSTACK_DUMP) {
    +            return jstackDumpCmd(workerPid, targetDir);
    +        } else if (action == ProfileAction.JPROFILE_DUMP) {
    +            return jprofileDump(workerPid, targetDir);
    +        } else if (action == ProfileAction.JVM_RESTART) {
    +            return jprofileJvmRestart(workerPid);
    +        } else if (!stop && action == ProfileAction.JPROFILE_STOP) {
    +            return jprofileStart(workerPid);
    +        } else if (stop && action == ProfileAction.JPROFILE_STOP) {
    +            return jprofileStop(workerPid, targetDir);
    +        }
    +        return Lists.newArrayList();
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +    
    +    protected String jlp(String stormRoot, Map<String, Object> conf) {
    +        String resourceRoot = stormRoot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +        String os = System.getProperty("os.name").replaceAll("\\s+", "_");
    +        String arch = System.getProperty("os.arch");
    +        String archResourceRoot = resourceRoot + Utils.FILE_PATH_SEPARATOR + os + "-" + arch;
    +        String ret = archResourceRoot + Utils.CLASS_PATH_SEPARATOR + resourceRoot + Utils.CLASS_PATH_SEPARATOR + conf.get(Config.JAVA_LIBRARY_PATH);
    +        return ret;
    +    }
    +    
    +    @SuppressWarnings("unchecked")
    +    protected String getWorkerClassPath(String stormJar, Map<String, Object> topologyConf) {
    +        List<String> topoClasspath = new ArrayList<>();
    +        Object object = topologyConf.get(Config.TOPOLOGY_CLASSPATH);
    +
    +        // Will be populated only if STORM_TOPOLOGY_CLASSPATH_BEGINNING_ENABLED is set on Nimbus.
    +        // Allowed for extreme debugging.
    +        Object topologyClasspathFirst = topologyConf.get(Config.TOPOLOGY_CLASSPATH_BEGINNING);
    +        List<String> firstClasspathList = new ArrayList<>();
    +        if(topologyClasspathFirst instanceof List) {
    +            firstClasspathList.addAll((List<String>)topologyClasspathFirst);
    +        } else if (topologyClasspathFirst instanceof String) {
    +            firstClasspathList.add((String) topologyClasspathFirst);
    +        }
    +        LOG.debug("Topology Classpath Prefix: {}", firstClasspathList);
    +
    +        if (object instanceof List) {
    +            topoClasspath.addAll((List<String>) object);
    +        } else if (object instanceof String) {
    +            topoClasspath.add((String) object);
    +        }
    +        LOG.debug("Topology specific classpath is {}", object);
    +
    +        String classPath = Utils.addToClasspath(firstClasspathList, Arrays.asList(Utils.workerClasspath()));
    +        String classAddPath = Utils.addToClasspath(classPath, Arrays.asList(stormJar));
    +        return Utils.addToClasspath(classAddPath, topoClasspath);
    +    }
    +    
    +    private static String substituteChildOptsInternal(String string,  String workerId, String stormId, int port, int memOnheap) {
    +        if (StringUtils.isNotBlank(string)){
    +            string = string.replace("%ID%", String.valueOf(port));
    +            string = string.replace("%WORKER-ID%", workerId);
    +            string = string.replace("%TOPOLOGY-ID%", stormId);
    +            string = string.replace("%WORKER-PORT%", String.valueOf(port));
    +            string = string.replace("%HEAP-MEM%", String.valueOf(memOnheap));
    +        }
    +        return string;
    +    }
    +    
    +    protected List<String> substituteChildopts(Object value, String workerId, String stormId, int port, int memOnheap) {
    +        List<String> rets = new ArrayList<>();
    +        if (value instanceof String) {
    +            String string = substituteChildOptsInternal((String) value,  workerId, stormId, port, memOnheap);
    +            if (StringUtils.isNotBlank(string)){
    +                String[] strings = string.split("\\s+");
    +                rets.addAll(Arrays.asList(strings));
    +            }
    +        } else if (value instanceof List) {
    +            @SuppressWarnings("unchecked")
    +            List<Object> objects = (List<Object>) value;
    +            for (Object object : objects) {
    +                String str = substituteChildOptsInternal((String) object,  workerId, stormId, port, memOnheap);
    +                if (StringUtils.isNotBlank(str)){
    +                    rets.add(str);
    +                }
    +            }
    +        }
    +        return rets;
    +    }
    +
    +    /**
    +     * Launch the worker process (non-blocking)
    +     * @param command the command to run
    +     * @param env the environment to run the command
    +     * @param processExitcallback a callback for when the process exits
    +     * @param logPrefix the prefix to include in the logs
    +     * @param targetDir the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException on any error
    +     */
    +    protected void launchWorkerProcess(List<String> command, Map<String, String> env, 
    +            String logPrefix, Utils.ExitCodeCallable<Void> processExitCallback, File targetDir) throws IOException {
    +        Utils.launchProcess(command, env, logPrefix, processExitCallback, targetDir);
    +    }
    +    
    +    @Override
    +    public void launch() throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot launch a container recovered with just a worker id");
    +        }
    +        final String topologyId = _assignment.get_topology_id();
    +        final WorkerResources resources = _assignment.get_resources();
    +
    +        LOG.info("Launching worker with assignment {} for this supervisor {} on port {} with id {}", _assignment, _supervisorId, _port,
    +                _workerId);
    +
    +        String logPrefix = "Worker Process " + _workerId;
    +
    +        ProcessExitCallback processExitCallback = new ProcessExitCallback(logPrefix);
    +        _exitedEarly = false;
    +
    +        String stormHome = ConfigUtils.concatIfNotNull(System.getProperty("storm.home"));
    --- End diff --
    
    we should make these strings as final string and keep it in Constants file



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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @HeartSaVioR Yes it looks like I need to think through recovery and any races with the AsyncLocalizer a bit more.  I'll try to reproduce your error.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @revans2 Thanks for the great work. Really appreciated. +1


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76608228
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/ResourceIsolationInterface.java ---
    @@ -56,4 +64,13 @@
          */
         List<String> getLaunchCommandPrefix(String workerId);
     
    +    /**
    +     * Get the list of PIDs currently in an isolated container
    +     * @param workerId the id of the worker to get these for
    +     * @return the set of PIDs, this will be combined with
    +     * other ways of getting PIDs. An Empty set or null if
    +     * no PIDs are found.
    +     * @throws IOException on any error
    +     */
    --- End diff --
    
    There is only one caller currently, and I thought because this is pluggable and the caller is not, that being more lenient here would be better, but I am happy to change it if you feel strongly about it.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78957510
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected final String _stormHome = System.getProperty("storm.home");
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +    
    +    /**
    +     * Create a new BasicContainer
    +     * @param type the type of container being made.
    +     * @param conf the supervisor config
    +     * @param supervisorId the ID of the supervisor this is a part of.
    +     * @param port the port the container is on.  Should be <= 0 if only a partial recovery
    +     * @param assignment the assignment for this container. Should be null if only a partial recovery.
    +     * @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used.
    +     * @param localState the local state of the supervisor.  May be null if partial recovery
    +     * @param workerId the id of the worker to use.  Must not be null if doing a partial recovery.
    +     * @param ops file system operations (mostly for testing) if null a new one is made
    +     * @param topoConf the config of the topology (mostly for testing) if null 
    +     * and not a partial recovery the real conf is read.
    +     * @param profileCmd the command to use when profiling (used for testing)
    +     */
    +    public BasicContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int port,
    +            LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
    +            LocalState localState, String workerId, Map<String, Object> topoConf, 
    +            AdvancedFSOps ops, String profileCmd) throws IOException {
    +        super(type, conf, supervisorId, port, assignment, resourceIsolationManager, workerId, topoConf, ops);
    +        assert(localState != null);
    +        _localState = localState;
    +
    +        if (type.isRecovery() && !type.isOnlyKillable()) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                LOG.info("Recovered Worker {}", wid);
    +                _workerId = wid;
    +            }
    +        } else if (_workerId == null){
    +            createNewWorkerId();
    +        }
    +
    +        if (profileCmd == null) {
    +            profileCmd = _stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                    + conf.get(Config.WORKER_PROFILER_COMMAND);
    +        }
    +        _profileCmd = profileCmd;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        _type.assertFull();
    +        assert(_workerId == null);
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            removeWorkersOn(workerToPort, _port);
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +            LOG.info("Created Worker ID {}", _workerId);
    +        }
    +    }
    +
    +    private static void removeWorkersOn(Map<String, Integer> workerToPort, int _port) {
    +        for (Iterator<Entry<String, Integer>> i = workerToPort.entrySet().iterator(); i.hasNext();) {
    +            Entry<String, Integer> found = i.next();
    +            if (_port == found.getValue().intValue()) {
    +                LOG.warn("Deleting worker {} from state", found.getKey());
    +                i.remove();
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUpForRestart() throws IOException {
    +        String origWorkerId = _workerId;
    +        super.cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(origWorkerId);
    +            removeWorkersOn(workersToPort, _port);
    +            _localState.setApprovedWorkers(workersToPort);
    +            LOG.info("Removed Worker ID {}", origWorkerId);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        _type.assertFull();
    +        //We are launching it now...
    +        _type = ContainerType.LAUNCH;
    +        createNewWorkerId();
    +        setup();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        _type.assertFull();
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        _type.assertFull();
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = _ops.slurpString(new File(str)).trim();
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        if (command.size() > 0) {
    +            return runProfilingCommand(command, env, logPrefix, targetFile);
    +        }
    +        LOG.warn("PROFILING REQUEST NOT SUPPORTED {} IGNORED...", request);
    +        return true;
    --- End diff --
    
    Shouldn't this return false because profiling didn't succeed?


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    I could not reproduce the one failure and the rat failure is fixed by STORM-2054 https://github.com/apache/storm/pull/1648



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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76462048
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,202 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.nio.file.FileSystems;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    +    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
    +    
    +    /**
    +     * Factory to create a new AdvancedFSOps
    +     * @param conf the configuration of the process
    +     * @return the appropriate instance of the class for this config and environment.
    +     */
    +    public static AdvancedFSOps mk(Map<String, Object> conf) {
    --- End diff --
    
    `mk` is a little terse for non-clojure. Can we change it to `make`?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79036299
  
    --- 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);
    +        
    +        //Ignore changes to scheduling while downloading the topology blobs
    +        // We don't support canceling the download through the future yet,
    +        // so to keep everything in sync, just wait
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization, staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        // We don't support canceling the download through the future yet,
    +        // so to keep everything in sync, just wait
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization, staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    --- End diff --
    
    Feel free to leave it. It shouldn't really matter.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77418682
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,420 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + * TODO once we have replaced the original supervisor merge this with
    + * Localizer and optimize them
    + */
    +public class AsyncLocalizer implements ILocalizer, Shutdownable {
    +    /**
    +     * A future that has already completed.
    +     */
    +    private static class AllDoneFuture implements Future<Void> {
    +
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isCancelled() {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isDone() {
    +            return true;
    +        }
    +
    +        @Override
    +        public Void get() {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void get(long timeout, TimeUnit unit) {
    +            return null;
    +        }
    +
    +    }
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizer.class);
    +
    +    private final Localizer _localizer;
    +    private final ExecutorService _execService;
    +    private final boolean _isLocalMode;
    +    private final Map<String, Object> _conf;
    +    private final Map<String, LocalDownloadedResource> _basicPending;
    +    private final Map<String, LocalDownloadedResource> _blobPending;
    +    private final AdvancedFSOps _fsOps;
    +
    +    private class DownloadBaseBlobsDistributed implements Callable<Void> {
    +        private final String _topologyId;
    +        
    +        public DownloadBaseBlobsDistributed(String topologyId) {
    +            this._topologyId = topologyId;
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            File sr = new File(stormroot);
    +            if (sr.exists()) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    Utils.forceDelete(stormroot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            try {
    +                String stormJarKey = ConfigUtils.masterStormJarKey(_topologyId);
    +                String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +                String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +                String jarPath = ConfigUtils.supervisorStormJarPath(tmproot);
    +                String codePath = ConfigUtils.supervisorStormCodePath(tmproot);
    +                String confPath = ConfigUtils.supervisorStormConfPath(tmproot);
    +                FileUtils.forceMkdir(new File(tmproot));
    +                _fsOps.restrictDirectoryPermissions(tmproot);
    +                ClientBlobStore blobStore = Utils.getClientBlobStoreForSupervisor(_conf);
    +                try {
    +                    Utils.downloadResourcesAsSupervisor(stormJarKey, jarPath, blobStore);
    +                    Utils.downloadResourcesAsSupervisor(stormCodeKey, codePath, blobStore);
    +                    Utils.downloadResourcesAsSupervisor(stormConfKey, confPath, blobStore);
    +                } finally {
    +                    blobStore.shutdown();
    +                }
    +                Utils.extractDirFromJar(jarPath, ConfigUtils.RESOURCES_SUBDIR, tmproot);
    +                _fsOps.moveDirectoryPreferAtomic(new File(tmproot), new File(stormroot));
    +                SupervisorUtils.setupStormCodeDir(_conf, ConfigUtils.readSupervisorStormConf(_conf, _topologyId), stormroot);
    +                deleteAll = false;
    +            } finally {
    +                if (deleteAll) {
    +                    LOG.info("Failed to download basic resources for topology-id {}", _topologyId);
    +                    Utils.forceDelete(tmproot);
    +                    Utils.forceDelete(stormroot);
    +                }
    +            }
    +            return null;
    +        }
    +    }
    +    
    +    private class DownloadBaseBlobsLocal implements Callable<Void> {
    +        private final String _topologyId;
    +        
    +        public DownloadBaseBlobsLocal(String topologyId) {
    +            this._topologyId = topologyId;
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            File sr = new File(stormroot);
    +            if (sr.exists()) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    Utils.forceDelete(stormroot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            try {
    +                BlobStore blobStore = Utils.getNimbusBlobStore(_conf, null, null);
    +                FileOutputStream codeOutStream = null;
    +                FileOutputStream confOutStream = null;
    +                try {
    +                    FileUtils.forceMkdir(new File(tmproot));
    +                    String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +                    String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +                    String codePath = ConfigUtils.supervisorStormCodePath(tmproot);
    +                    String confPath = ConfigUtils.supervisorStormConfPath(tmproot);
    +                    codeOutStream = new FileOutputStream(codePath);
    +                    blobStore.readBlobTo(stormCodeKey, codeOutStream, null);
    +                    confOutStream = new FileOutputStream(confPath);
    +                    blobStore.readBlobTo(stormConfKey, confOutStream, null);
    +                } finally {
    +                    if (codeOutStream != null)
    +                        codeOutStream.close();
    +                    if (confOutStream != null)
    +                        codeOutStream.close();
    +                    blobStore.shutdown();
    +                }
    +
    +                ClassLoader classloader = Thread.currentThread().getContextClassLoader();
    +                String resourcesJar = AsyncLocalizer.resourcesJar();
    +                URL url = classloader.getResource(ConfigUtils.RESOURCES_SUBDIR);
    +
    +                String targetDir = tmproot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +
    +                if (resourcesJar != null) {
    +                    LOG.info("Extracting resources from jar at {} to {}", resourcesJar, targetDir);
    +                    Utils.extractDirFromJar(resourcesJar, ConfigUtils.RESOURCES_SUBDIR, stormroot);
    +                } else if (url != null) {
    +                    LOG.info("Copying resources at {} to {} ", url.toString(), targetDir);
    +                    if (url.getProtocol() == "jar") {
    --- End diff --
    
    `.equals` here?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78971109
  
    --- 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);
    +        
    --- End diff --
    
    You could assert that areAllProcessesDead is true here


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76474197
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,437 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = -1;
    +        _assignment = null;
    +        _workerId = workerId;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq"+pid);
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            if (in.readLine() != null) {
    +                ret = true;
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    private boolean isPosixProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("ps", "-o", "user", "-p", String.valueOf(pid));
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            String first = in.readLine();
    +            assert("USER".equals(first));
    +            String processUser;
    +            while ((processUser = in.readLine()) != null) {
    +                if (user.equals(processUser)) {
    +                    ret = true;
    +                    break;
    +                } else {
    +                    LOG.info("Found {} running as {}, but expected it to be {}", pid, processUser, user);
    +                }
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    @Override
    +    public boolean areAllProcessesDead() throws IOException {
    +        Set<Long> pids = getAllPids();
    +        String user = getWorkerUser();
    +        
    +        boolean allDead = true;
    +        for (Long pid: pids) {
    +            if (!isProcessAlive(pid, user)) {
    +                LOG.warn("{}: PID {} is dead", _workerId, pid);
    +            } else {
    +                allDead = false;
    +                break;
    +            }
    +        }
    +        return allDead;
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +    }
    +
    +    /**
    +     * Setup the container to run.  By default this creates the needed directories/links in the
    +     * local file system
    +     * PREREQUISITE: All needed blobs and topology, jars/configs have been downloaded and
    +     * placed in the appropriate locations
    +     * @throws IOException on any error
    +     */
    +    protected void setup() throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    --- End diff --
    
    Could we clarify this a bit? I wasn't exactly sure what this meant with this wording.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @HeartSaVioR I found the original comment and put it the change.  I also fixed some other review comments from @abellina and also fixed an annoying failure with drpc_auth_test.clj not being able to bind to a port.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    I'm still reviewing this. Nice work to refactor to introduce greater readability. Will leave comment here once I'm done with first pass.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76473887
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,437 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = -1;
    +        _assignment = null;
    +        _workerId = workerId;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq"+pid);
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            if (in.readLine() != null) {
    +                ret = true;
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    private boolean isPosixProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("ps", "-o", "user", "-p", String.valueOf(pid));
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            String first = in.readLine();
    +            assert("USER".equals(first));
    +            String processUser;
    +            while ((processUser = in.readLine()) != null) {
    +                if (user.equals(processUser)) {
    +                    ret = true;
    +                    break;
    +                } else {
    +                    LOG.info("Found {} running as {}, but expected it to be {}", pid, processUser, user);
    +                }
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    @Override
    +    public boolean areAllProcessesDead() throws IOException {
    +        Set<Long> pids = getAllPids();
    +        String user = getWorkerUser();
    +        
    +        boolean allDead = true;
    +        for (Long pid: pids) {
    +            if (!isProcessAlive(pid, user)) {
    +                LOG.warn("{}: PID {} is dead", _workerId, pid);
    --- End diff --
    
    debug?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77353587
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    --- End diff --
    
    very minor: This line is long and can be shorted by using <>  for the AtomicReference and for the HashMap.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77350298
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,484 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.io.Reader;
    +import java.io.Writer;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected final Map<String, Object> _topoConf;
    +    protected String _workerId;
    +    protected final String _topologyId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    //Exposed for testing
    +    protected Container(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        assert((assignment == null && port <= 0) ||
    +                (assignment != null && port > 0));
    +        assert(conf != null);
    +        assert(ops != null);
    +        assert(supervisorId != null);
    +        
    +        _port = port;
    +        _ops = ops;
    +        _assignment = assignment;
    +        if (assignment != null) {
    +            _topologyId = assignment.get_topology_id();
    +        } else {
    +            _topologyId = null;
    +        }
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        if (topoConf == null) {
    +            _topoConf = readTopoConf();
    +        } else {
    +            _topoConf = topoConf;
    +        }
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return this.getClass().getSimpleName() + " topo:" + _topologyId + " worker:" + _workerId;
    +    }
    +    
    +    protected Map<String, Object> readTopoConf() throws IOException {
    +        assert(_topologyId != null);
    +        return ConfigUtils.readSupervisorStormConf(_conf, _topologyId);
    +    }
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), port, assignment, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     * @throws IOException on any error
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), -1, null, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.trace("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq" + pid);
    --- End diff --
    
    OK, I was not clear in my earlier comment. I didn't mean we needed spaces around the `+` but that we need one within the quoted string after the `eq`: `"pid eq " + pid`


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77323762
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/SupervisorUtils.java ---
    @@ -159,7 +162,7 @@ public static void addBlobReferences(Localizer localizer, String stormId, Map co
             }
         }
     
    -    public static Set<String> readDownLoadedStormIds(Map conf) throws IOException {
    +    public static Set<String> readDownLoadedStormIds(Map<String, Object> conf) throws IOException {
             Set<String> stormIds = new HashSet<>();
             String path = ConfigUtils.supervisorStormDistRoot(conf);
             Collection<String> rets = Utils.readDirContents(path);
    --- End diff --
    
    This is not a bug from the patch, but in OSX .DS_Store file can be created, so we need to filter out `files` in rets and keep only `directories`.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76272495
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,749 @@
    +/**
    + * 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,
    +        WATING_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 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 monitorFreqMs,
    +                ContainerLauncher containerLauncher, String host, int port,
    +                ISupervisor iSupervisor, LocalState localState) {
    +            this.localizer = localizer;
    +            this.hbTimeoutMs = hbTimeoutMs;
    +            this.firstHbTimeoutMs = firstHbTimeoutMs;
    +            this.containerLauncher = containerLauncher;
    +            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 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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WATING_FOR_WORKER_START) {
    +                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 equivilant(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;
    +                    } else 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 WATING_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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), staticState.port);
    +        }
    +        Time.sleep(1000);
    --- End diff --
    
    Any significance to sleep here. we should add a comment and make it config option.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77412935
  
    --- 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);
    +            }
    +            if (container != null) {
    +                sb.append(" container: ");
    +                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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WAITING_FOR_WORKER_START) {
    +                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;
    +                    } else 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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), 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.get_topology_id(), 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);
    +        
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        //The child processes typically exit in < 1 sec.  If 2 mins later they are still around something is wrong
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for WAITING_FOR_WORKER_START state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleWaitingForWorkerStart(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb != null) {
    +            long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +            if (hbAgeMs <= staticState.hbTimeoutMs) {
    +                return dynamicState.withState(MachineState.RUNNING);
    +            }
    +        }
    +        
    +        if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            //We were rescheduled while waiting for the worker to come up
    +            return Slot.killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffms = (Time.currentTimeMillis() - dynamicState.startTime);
    +        if (timeDiffms > staticState.firstHbTimeoutMs) {
    +            LOG.warn("SLOT {}: Container {} failed to launch in {} ms.", staticState.port, dynamicState.container, staticState.firstHbTimeoutMs);
    +            dynamicState.container.kill();
    +            Time.sleep(staticState.killSleepMs);
    +            return dynamicState.withState(MachineState.KILL_AND_RELAUNCH);
    --- End diff --
    
    We should just call `killAndRelaunchContainer` here. In this transition, there should not be any profiler actions as it has not come up yet, so that extra setup is a NOOP.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79030558
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected final String _stormHome = System.getProperty("storm.home");
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +    
    +    /**
    +     * Create a new BasicContainer
    +     * @param type the type of container being made.
    +     * @param conf the supervisor config
    +     * @param supervisorId the ID of the supervisor this is a part of.
    +     * @param port the port the container is on.  Should be <= 0 if only a partial recovery
    +     * @param assignment the assignment for this container. Should be null if only a partial recovery.
    +     * @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used.
    +     * @param localState the local state of the supervisor.  May be null if partial recovery
    +     * @param workerId the id of the worker to use.  Must not be null if doing a partial recovery.
    +     * @param ops file system operations (mostly for testing) if null a new one is made
    +     * @param topoConf the config of the topology (mostly for testing) if null 
    +     * and not a partial recovery the real conf is read.
    +     * @param profileCmd the command to use when profiling (used for testing)
    +     */
    +    public BasicContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int port,
    +            LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
    +            LocalState localState, String workerId, Map<String, Object> topoConf, 
    +            AdvancedFSOps ops, String profileCmd) throws IOException {
    +        super(type, conf, supervisorId, port, assignment, resourceIsolationManager, workerId, topoConf, ops);
    +        assert(localState != null);
    +        _localState = localState;
    +
    +        if (type.isRecovery() && !type.isOnlyKillable()) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                LOG.info("Recovered Worker {}", wid);
    +                _workerId = wid;
    +            }
    +        } else if (_workerId == null){
    +            createNewWorkerId();
    +        }
    +
    +        if (profileCmd == null) {
    +            profileCmd = _stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                    + conf.get(Config.WORKER_PROFILER_COMMAND);
    +        }
    +        _profileCmd = profileCmd;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        _type.assertFull();
    +        assert(_workerId == null);
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            removeWorkersOn(workerToPort, _port);
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +            LOG.info("Created Worker ID {}", _workerId);
    +        }
    +    }
    +
    +    private static void removeWorkersOn(Map<String, Integer> workerToPort, int _port) {
    +        for (Iterator<Entry<String, Integer>> i = workerToPort.entrySet().iterator(); i.hasNext();) {
    +            Entry<String, Integer> found = i.next();
    +            if (_port == found.getValue().intValue()) {
    +                LOG.warn("Deleting worker {} from state", found.getKey());
    +                i.remove();
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUpForRestart() throws IOException {
    +        String origWorkerId = _workerId;
    +        super.cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(origWorkerId);
    +            removeWorkersOn(workersToPort, _port);
    +            _localState.setApprovedWorkers(workersToPort);
    +            LOG.info("Removed Worker ID {}", origWorkerId);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        _type.assertFull();
    +        //We are launching it now...
    +        _type = ContainerType.LAUNCH;
    +        createNewWorkerId();
    +        setup();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        _type.assertFull();
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        _type.assertFull();
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = _ops.slurpString(new File(str)).trim();
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        if (command.size() > 0) {
    +            return runProfilingCommand(command, env, logPrefix, targetFile);
    +        }
    +        LOG.warn("PROFILING REQUEST NOT SUPPORTED {} IGNORED...", request);
    +        return true;
    --- End diff --
    
    This is a special case.  It essentially means that someone created a profiling request that we don't know what to do with.  This could be because of a rolling upgrade or something like that, but somehow nimbus did something we don't expect.
    
    Returning false will cause the command to be retried, but at this point the supervisor has no idea what to do with the profiling request and the original code was ignoring these, so I decided to do the same.  If we want to change it we can, but I would rather do that in a follow on JIRA.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79031226
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,328 @@
    +/**
    + * 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.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 java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +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.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    --- End diff --
    
    I was originally trying to follow the convention in the file itself.  This came from Supervisor.java that didn't do it and was split off from that as part of the review process.  I personally like the _ convention, but am OK with whatever.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76991771
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,319 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.Writer;
    +import java.nio.file.FileSystems;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    +    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
    +    
    +    /**
    +     * Factory to create a new AdvancedFSOps
    +     * @param conf the configuration of the process
    +     * @return the appropriate instance of the class for this config and environment.
    +     */
    +    public static AdvancedFSOps make(Map<String, Object> conf) {
    +        if (Utils.isOnWindows()) {
    +            return new AdvancedWindowsFSOps(conf);
    +        }
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new AdvancedRunAsUserFSOps(conf);
    +        }
    +        return new AdvancedFSOps();
    +    }
    +    
    +    private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
    +        private final Map<String, Object> _conf;
    +        
    +        public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
    +            if (Utils.isOnWindows()) {
    +                throw new UnsupportedOperationException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +            _conf = conf;
    +        }
    +        
    +        @Override
    +        public void setupBlobPermissions(File path, String user) throws IOException {
    +            String logPrefix = "setup blob permissions for " + path;
    +            SupervisorUtils.processLauncherAndWait(_conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
    +        }
    +        
    +        @Override
    +        public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +            String absolutePath = path.getAbsolutePath();
    +            LOG.info("Deleting path {}", absolutePath);
    +            if (user == null) {
    +                user = Files.getOwner(path.toPath()).getName();
    +            }
    +            List<String> commands = new ArrayList<>();
    +            commands.add("rmr");
    +            commands.add(absolutePath);
    +            SupervisorUtils.processLauncherAndWait(_conf, user, commands, null, logPrefix);
    +            if (Utils.checkFileExists(absolutePath)) {
    +                throw new RuntimeException(path + " was not deleted.");
    +            }
    +        }
    +        
    +        @Override
    +        public void setupStormCodeDir(Map<String, Object> topologyConf, File path) throws IOException {
    +            SupervisorUtils.setupStormCodeDir(_conf, topologyConf, path.getCanonicalPath());
    +        }
    +    }
    +    
    +    /**
    +     * Operations that need to override the default ones when running on Windows
    +     *
    +     */
    +    private static class AdvancedWindowsFSOps extends AdvancedFSOps {
    +
    +        public AdvancedWindowsFSOps(Map<String, Object> conf) {
    +            if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +        }
    +        
    +        @Override
    +        public void restrictDirectoryPermissions(String dir) throws IOException {
    +            //NOOP, if windows gets support for run as user we will need to find a way to suppor this
    +        }
    +        
    +        @Override
    +        public void moveDriectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // This is not atomic but it does work
    +            FileUtils.moveDirectory(fromDir, toDir);
    +        }
    +        
    +        @Override
    +        public boolean supportsAtomicDirectoryMove() {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // FileUtils.moveDirectory is not atomic
    +            return false;
    +        }
    +    }
    +    
    +    
    +    protected AdvancedFSOps() {
    +        //NOOP, but restricted permissions
    +    }
    +
    +    /**
    +     * Set directory permissions to (OWNER)RWX (GROUP)R-X (OTHER)---
    +     * On some systems that do not support this, it may become a noop
    +     * @param dir the directory to change permissions on
    +     * @throws IOException on any error
    +     */
    +    public void restrictDirectoryPermissions(String dir) throws IOException {
    +        Set<PosixFilePermission> perms = new HashSet<>(
    +                Arrays.asList(PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE,
    +                        PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ,
    +                        PosixFilePermission.GROUP_EXECUTE));
    +        Files.setPosixFilePermissions(FileSystems.getDefault().getPath(dir), perms);
    +    }
    +
    +    /**
    +     * Move fromDir to toDir, and try to make it an atomic move if possible
    +     * @param fromDir what to move
    +     * @param toDir where to move it from
    +     * @throws IOException on any error
    +     */
    +    public void moveDriectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +        FileUtils.forceMkdir(toDir);
    +        Files.move(fromDir.toPath(), toDir.toPath(), StandardCopyOption.ATOMIC_MOVE);
    +    }
    +    
    +    /**
    +     * @return true if an atomic directory move works, else false.
    +     */
    +    public boolean supportsAtomicDirectoryMove() {
    +        return true;
    +    }
    +    
    +    /**
    +     * Setup permissions properly for an internal blob store path
    +     * @param path the path to set the permissions on
    +     * @param user the user to change the permissions for
    +     * @throws IOException on any error
    +     */
    +    public void setupBlobPermissions(File path, String user) throws IOException {
    +        //Normally this is a NOOP
    +    }
    +
    +    /**
    +     * Delete a file or a directory and all of the children. If it exists.
    +     * @param path what to delete
    +     * @param user who to delete it as if doing it as someone else is supported
    +     * @param logPrefix if an external process needs to be launched to delete 
    +     * the object what prefix to include in the logs
    +     * @throws IOException on any error.
    +     */
    +    public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +        LOG.info("Deleting path {}", path);
    --- End diff --
    
    We could just call deleteIfExists(path) since it's same.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79032466
  
    --- 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);
    --- End diff --
    
    The state itself is not necessarily empty.  It could be that we just killed all of the workers.  I'll rename the function to avoid confusion.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77260851
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +
    +    //For testing purposes
    +    public BasicContainer(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager, LocalState localState,
    +            String profileCmd) throws IOException {
    +        super(ops, port, assignment, conf, topoConf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +        _profileCmd = profileCmd;
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState localState, ResourceIsolationInterface resourceIsolationManager, boolean recover)
    +            throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                LOG.info("Recovered Worker {}", wid);
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    --- End diff --
    
    done


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78959596
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,328 @@
    +/**
    + * 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.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 java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +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.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    --- End diff --
    
    Nitpick: Why do some files prefix fields with _ and this one doesn't?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77361623
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/RunAsUserContainer.java ---
    @@ -0,0 +1,107 @@
    +/**
    + * 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.File;
    +import java.io.IOException;
    +import java.util.Arrays;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class RunAsUserContainer extends BasicContainer {
    +    private static final Logger LOG = LoggerFactory.getLogger(RunAsUserContainer.class);
    +
    +    public RunAsUserContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState state, ResourceIsolationInterface resourceIsolationManager, boolean recover) throws IOException {
    +        super(port, assignment, conf, supervisorId, state, resourceIsolationManager, recover);
    +        if (Utils.isOnWindows()) {
    +            throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    --- End diff --
    
    UnsupportedOperationException


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76479982
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ContainerLauncher.java ---
    @@ -0,0 +1,99 @@
    +/**
    + * 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.Map;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Launches containers
    + */
    +public abstract class ContainerLauncher {
    +    private static final Logger LOG = LoggerFactory.getLogger(ContainerLauncher.class);
    +    
    +    /**
    +     * Factory to create the right container launcher 
    +     * for the config and the environment.
    +     * @param conf the config
    +     * @param supervisorId the ID of the supervisor
    +     * @param sharedContext Used in local mode to let workers talk together without netty
    +     * @return the proper container launcher
    +     * @throws IOException on any error
    +     */
    +    public static ContainerLauncher mk(Map<String, Object> conf, String supervisorId, IContext sharedContext) throws IOException {
    +        if (ConfigUtils.isLocalMode(conf)) {
    +            return new LocalContainerLauncher(conf, supervisorId, sharedContext);
    +        }
    +        
    +        ResourceIsolationInterface resourceIsolationManager = null;
    +        if (Utils.getBoolean(conf.get(Config.STORM_RESOURCE_ISOLATION_PLUGIN_ENABLE), false)) {
    +            resourceIsolationManager = Utils.newInstance((String) conf.get(Config.STORM_RESOURCE_ISOLATION_PLUGIN));
    +            resourceIsolationManager.prepare(conf);
    +            LOG.info("Using resource isolation plugin {} {}", conf.get(Config.STORM_RESOURCE_ISOLATION_PLUGIN), resourceIsolationManager);
    +        }
    +
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new RunAsUserContainerLauncher(conf, supervisorId, resourceIsolationManager);
    +        }
    +        return new BasicContainerLauncher(conf, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    protected ContainerLauncher() {
    +        //Empty
    +    }
    +
    +    /**
    +     * Launch a container in a given slot
    +     * @param port the port to run this on
    +     * @param assignment what to launch
    +     * @param state the current state of the supervisor
    +     * @return The container that can be used to manager the processes.
    +     * @throws IOException on any error 
    +     */
    +    public abstract Container launchContainer(int port, LocalAssignment assignment, LocalState state) throws IOException;
    +    
    +    /**
    +     * Recover a container for a running process
    +     * @param port the port the assignment is running on
    +     * @param assignment the assignment that was launched
    +     * @param state the current state of the supervisor
    +     * @return The container that can be used to manage the processes.
    +     * @throws IOException on any error
    +     */
    +    public abstract Container recoverContainer(int port, LocalAssignment assignment, LocalState state) throws IOException;
    +    
    +    /**
    +     * Try to recover a container with just the worker ID.  
    +     * The result is really only useful for killing the container.
    +     * @param workerId the id of the worker to use
    +     * @return a Container that can be killed.
    +     * @throws IOException on any error
    +     */
    +    public abstract Killable recoverContainer(String workerId) throws IOException;
    --- End diff --
    
    I like the idea of using an interface specific to the scenario. Only comment is we should update the javadoc to say it returns a Killable instead of a Container.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78950399
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected final String _stormHome = System.getProperty("storm.home");
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +    
    +    /**
    +     * Create a new BasicContainer
    +     * @param type the type of container being made.
    +     * @param conf the supervisor config
    +     * @param supervisorId the ID of the supervisor this is a part of.
    +     * @param port the port the container is on.  Should be <= 0 if only a partial recovery
    +     * @param assignment the assignment for this container. Should be null if only a partial recovery.
    +     * @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used.
    +     * @param localState the local state of the supervisor.  May be null if partial recovery
    +     * @param workerId the id of the worker to use.  Must not be null if doing a partial recovery.
    +     * @param ops file system operations (mostly for testing) if null a new one is made
    +     * @param topoConf the config of the topology (mostly for testing) if null 
    +     * and not a partial recovery the real conf is read.
    +     * @param profileCmd the command to use when profiling (used for testing)
    +     */
    +    public BasicContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int port,
    +            LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
    +            LocalState localState, String workerId, Map<String, Object> topoConf, 
    +            AdvancedFSOps ops, String profileCmd) throws IOException {
    +        super(type, conf, supervisorId, port, assignment, resourceIsolationManager, workerId, topoConf, ops);
    +        assert(localState != null);
    +        _localState = localState;
    +
    +        if (type.isRecovery() && !type.isOnlyKillable()) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    --- End diff --
    
    Remember to log this at some level, the exception gets swallowed further up. Alternatively, maybe the launchers should log caught ContainerRecoveryExceptions.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78952788
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    --- End diff --
    
    Nitpick: FilenameFilter is a functional interface. You can replace the anonymous class with a lambda if you prefer.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76478540
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ContainerLauncher.java ---
    @@ -0,0 +1,99 @@
    +/**
    + * 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.Map;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Launches containers
    + */
    +public abstract class ContainerLauncher {
    +    private static final Logger LOG = LoggerFactory.getLogger(ContainerLauncher.class);
    +    
    +    /**
    +     * Factory to create the right container launcher 
    +     * for the config and the environment.
    +     * @param conf the config
    +     * @param supervisorId the ID of the supervisor
    +     * @param sharedContext Used in local mode to let workers talk together without netty
    +     * @return the proper container launcher
    +     * @throws IOException on any error
    +     */
    +    public static ContainerLauncher mk(Map<String, Object> conf, String supervisorId, IContext sharedContext) throws IOException {
    --- End diff --
    
    Rename `make`?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77245653
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,314 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.Writer;
    +import java.nio.file.FileSystems;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    --- End diff --
    
    AdvancedFSOps is the default only if you are on windows or if you are doing run as user is a subclass used.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78969293
  
    --- 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);
    --- End diff --
    
    Maybe there should be an assertion that slot state is empty?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77407551
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,493 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.io.Reader;
    +import java.io.Writer;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected final Map<String, Object> _topoConf;
    +    protected String _workerId;
    +    protected final String _topologyId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    //Exposed for testing
    +    protected Container(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        assert((assignment == null && port <= 0) ||
    +                (assignment != null && port > 0));
    +        assert(conf != null);
    +        assert(ops != null);
    +        assert(supervisorId != null);
    +        
    +        _port = port;
    +        _ops = ops;
    +        _assignment = assignment;
    +        if (assignment != null) {
    +            _topologyId = assignment.get_topology_id();
    +        } else {
    +            _topologyId = null;
    +        }
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        if (topoConf == null) {
    +            _topoConf = readTopoConf();
    +        } else {
    +            _topoConf = topoConf;
    +        }
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return this.getClass().getSimpleName() + " topo:" + _topologyId + " worker:" + _workerId;
    +    }
    +    
    +    protected Map<String, Object> readTopoConf() throws IOException {
    +        assert(_topologyId != null);
    +        return ConfigUtils.readSupervisorStormConf(_conf, _topologyId);
    +    }
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), port, assignment, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     * @throws IOException on any error
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), -1, null, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.trace("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    --- End diff --
    
    Parsing this out without a working windows box is more then I am comfortable with.  If someone else wants to write it and make a pull request to this branch I am happy to accept it.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76823287
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/LocalDownloadedResource.java ---
    @@ -0,0 +1,107 @@
    +/**
    + * 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.localizer;
    +
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class LocalDownloadedResource {
    +    private static final Logger LOG = LoggerFactory.getLogger(LocalDownloadedResource.class);
    +    private static class NoCancelFuture<T> implements Future<T> {
    +        private final Future<T> _wrapped;
    +        
    +        public NoCancelFuture(Future<T> wrapped) {
    +            _wrapped = wrapped;
    +        }
    +        
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            //cancel not currently supported
    --- End diff --
    
    currently or never? (given that this is the NoCancel future)


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @HeartSaVioR do we want this for 1.x branch? Given that it's a java migration and pretty much a new feature that can get into 1.x-branch. I would rather keep 1.x-branch as it is.
    @revans2 will go through it today.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78968525
  
    --- 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.
    --- End diff --
    
    Nitpick: "my" should be "by"


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76459256
  
    --- Diff: storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java ---
    @@ -164,21 +164,18 @@ public Assignment assignmentInfo(String stormId, Runnable callback) {
         }
     
         @Override
    -    public Map assignmentInfoWithVersion(String stormId, Runnable callback) {
    -        Map map = new HashMap();
    +    public VersionedData<Assignment> assignmentInfoWithVersion(String stormId, Runnable callback) {
             if (callback != null) {
                 assignmentInfoWithVersionCallback.put(stormId, callback);
             }
             Assignment assignment = null;
             Integer version = 0;
    -        Map dataWithVersionMap = stateStorage.get_data_with_version(ClusterUtils.assignmentPath(stormId), callback != null);
    -        if (dataWithVersionMap != null) {
    -            assignment = ClusterUtils.maybeDeserialize((byte[]) dataWithVersionMap.get(IStateStorage.DATA), Assignment.class);
    -            version = (Integer) dataWithVersionMap.get(IStateStorage.VERSION);
    -        }
    -        map.put(IStateStorage.DATA, assignment);
    -        map.put(IStateStorage.VERSION, version);
    -        return map;
    +        VersionedData<byte[]> dataWithVersion = stateStorage.get_data_with_version(ClusterUtils.assignmentPath(stormId), callback != null);
    +        if (dataWithVersion != null) {
    +            assignment = ClusterUtils.maybeDeserialize((byte[]) dataWithVersion.getData(), Assignment.class);
    --- End diff --
    
    VersionedData is generic, so we can remove the cast to byte[]


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76535617
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,300 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.Writer;
    +import java.nio.file.FileSystems;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.Paths;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    +    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
    +    
    +    /**
    +     * Factory to create a new AdvancedFSOps
    +     * @param conf the configuration of the process
    +     * @return the appropriate instance of the class for this config and environment.
    +     */
    +    public static AdvancedFSOps mk(Map<String, Object> conf) {
    +        if (Utils.isOnWindows()) {
    +            return new AdvancedWindowsFSOps(conf);
    +        }
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new AdvancedRunAsUserFSOps(conf);
    +        }
    +        return new AdvancedFSOps();
    +    }
    +    
    +    private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
    +        private final Map<String, Object> _conf;
    +        
    +        public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
    +            if (Utils.isOnWindows()) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +            _conf = conf;
    +        }
    +        
    +        @Override
    +        public void setupBlobPermissions(File path, String user) throws IOException {
    +            String logPrefix = "setup blob permissions for " + path;
    +            SupervisorUtils.processLauncherAndWait(_conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
    +        }
    +        
    +        @Override
    +        public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +            String absolutePath = path.getAbsolutePath();
    +            LOG.debug("Deleting path {}", absolutePath);
    +            if (user == null) {
    +                user = Files.getOwner(path.toPath()).getName();
    +            }
    +            List<String> commands = new ArrayList<>();
    +            commands.add("rmr");
    +            commands.add(absolutePath);
    +            SupervisorUtils.processLauncherAndWait(_conf, user, commands, null, logPrefix);
    +            if (Utils.checkFileExists(absolutePath)) {
    +                throw new RuntimeException(path + " was not deleted.");
    +            }
    +        }
    +        
    +        @Override
    +        public void setupStormCodeDir(Map<String, Object> topologyConf, File path) throws IOException {
    +            SupervisorUtils.setupStormCodeDir(_conf, topologyConf, path.getCanonicalPath());
    +        }
    +    }
    +    
    +    /**
    +     * Operations that need to override the default ones when running on Windows
    +     *
    +     */
    +    private static class AdvancedWindowsFSOps extends AdvancedFSOps {
    +
    +        public AdvancedWindowsFSOps(Map<String, Object> conf) {
    +            if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +        }
    +        
    +        @Override
    +        public void restrictDirectoryPermissions(String dir) throws IOException {
    +            //NOOP, if windows gets support for run as user we will need to find a way to suppor this
    +        }
    +        
    +        @Override
    +        public void moveDriectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // This is not atomic but it does work
    +            FileUtils.moveDirectory(fromDir, toDir);
    +        }
    +        
    +        @Override
    +        public boolean supportsAtomicDirectoryMove() {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // FileUtils.moveDirectory is not atomic
    +            return false;
    +        }
    +    }
    +    
    +    
    +    protected AdvancedFSOps() {
    +        //NOOP, but restricted permissions
    +    }
    +
    +    /**
    +     * Set directory permissions to (OWNER)RWX (GROUP)R-X (OTHER)---
    +     * On some systems that do not support this, it may become a noop
    +     * @param dir the directory to change permissions on
    +     * @throws IOException on any error
    +     */
    +    public void restrictDirectoryPermissions(String dir) throws IOException {
    +        Set<PosixFilePermission> perms = new HashSet<>(
    +                Arrays.asList(PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE,
    +                        PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ,
    +                        PosixFilePermission.GROUP_EXECUTE));
    +        Files.setPosixFilePermissions(FileSystems.getDefault().getPath(dir), perms);
    +    }
    +
    +    /**
    +     * Move fromDir to toDir, and try to make it an atomic move if possible
    +     * @param fromDir what to move
    +     * @param toDir where to move it from
    +     * @throws IOException on any error
    +     */
    +    public void moveDriectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +        FileUtils.forceMkdir(toDir);
    +        Files.move(fromDir.toPath(), toDir.toPath(), StandardCopyOption.ATOMIC_MOVE);
    +    }
    +    
    +    /**
    +     * @return true if an atomic directory move works, else false.
    +     */
    +    public boolean supportsAtomicDirectoryMove() {
    +        return true;
    +    }
    +    
    +    /**
    +     * Setup permissions properly for an internal blob store path
    +     * @param path the path to the permissions
    +     * @param user the user to change the permissions for
    +     * @throws IOException on any error
    +     */
    +    public void setupBlobPermissions(File path, String user) throws IOException {
    +        //Normally this is a NOOP
    +    }
    +
    +    /**
    +     * Delete a file or a directory and all of the children. If it exists.
    +     * @param path what to delete
    +     * @param user who to delete it as if doing it as someone else is supported
    +     * @param logPrefix if an external process needs to be launched to delete 
    +     * the object what prefix to include in the logs
    +     * @throws IOException on any error.
    +     */
    +    public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +        LOG.debug("Deleting path {}", path);
    --- End diff --
    
    this method doesn't use the user or logPrefix. It also does the same thing as the overload below.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76334398
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,202 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.nio.file.FileSystems;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    +    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
    +    
    +    /**
    +     * Factory to create a new AdvancedFSOps
    +     * @param conf the configuration of the process
    +     * @return the appropriate instance of the class for this config and environment.
    +     */
    +    public static AdvancedFSOps mk(Map<String, Object> conf) {
    +        if (Utils.isOnWindows()) {
    +            return new AdvancedWindowsFSOps(conf);
    +        }
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new AdvancedRunAsUserFSOps(conf);
    +        }
    +        return new AdvancedFSOps();
    +    }
    +    
    +    private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
    +        private final Map<String, Object> _conf;
    +        
    +        public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
    +            if (Utils.isOnWindows()) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +            _conf = conf;
    +        }
    +        
    +        @Override
    +        public void setupBlobPermissions(File path, String user) throws IOException {
    +            String logPrefix = "setup blob permissions for " + path;
    +            SupervisorUtils.processLauncherAndWait(_conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
    +        }
    +        
    +        public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    --- End diff --
    
    need Override tag


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79037939
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,422 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.net.URLDecoder;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + */
    +public class AsyncLocalizer implements ILocalizer, Shutdownable {
    +    /**
    +     * A future that has already completed.
    +     */
    +    private static class AllDoneFuture implements Future<Void> {
    +
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isCancelled() {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isDone() {
    +            return true;
    +        }
    +
    +        @Override
    +        public Void get() {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void get(long timeout, TimeUnit unit) {
    +            return null;
    +        }
    +
    +    }
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizer.class);
    +
    +    private final Localizer _localizer;
    +    private final ExecutorService _execService;
    +    private final boolean _isLocalMode;
    +    private final Map<String, Object> _conf;
    +    private final Map<String, LocalDownloadedResource> _basicPending;
    +    private final Map<String, LocalDownloadedResource> _blobPending;
    +    private final AdvancedFSOps _fsOps;
    +
    +    private class DownloadBaseBlobsDistributed implements Callable<Void> {
    +        protected final String _topologyId;
    +        protected final File _stormRoot;
    +        
    +        public DownloadBaseBlobsDistributed(String topologyId) throws IOException {
    +            _topologyId = topologyId;
    +            _stormRoot = new File(ConfigUtils.supervisorStormDistRoot(_conf, _topologyId));
    +        }
    +        
    +        protected void downloadBaseBlobs(File tmproot) throws Exception {
    +            String stormJarKey = ConfigUtils.masterStormJarKey(_topologyId);
    +            String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +            String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +            String jarPath = ConfigUtils.supervisorStormJarPath(tmproot.getAbsolutePath());
    +            String codePath = ConfigUtils.supervisorStormCodePath(tmproot.getAbsolutePath());
    +            String confPath = ConfigUtils.supervisorStormConfPath(tmproot.getAbsolutePath());
    +            _fsOps.forceMkdir(tmproot);
    +            _fsOps.restrictDirectoryPermissions(tmproot);
    +            ClientBlobStore blobStore = Utils.getClientBlobStoreForSupervisor(_conf);
    +            try {
    +                Utils.downloadResourcesAsSupervisor(stormJarKey, jarPath, blobStore);
    +                Utils.downloadResourcesAsSupervisor(stormCodeKey, codePath, blobStore);
    +                Utils.downloadResourcesAsSupervisor(stormConfKey, confPath, blobStore);
    +            } finally {
    +                blobStore.shutdown();
    +            }
    +            Utils.extractDirFromJar(jarPath, ConfigUtils.RESOURCES_SUBDIR, tmproot);
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            if (_fsOps.fileExists(_stormRoot)) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    _fsOps.deleteIfExists(_stormRoot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            File tr = new File(tmproot);
    +            try {
    +                downloadBaseBlobs(tr);
    +                _fsOps.moveDirectoryPreferAtomic(tr, _stormRoot);
    +                _fsOps.setupStormCodeDir(ConfigUtils.readSupervisorStormConf(_conf, _topologyId), _stormRoot);
    +                deleteAll = false;
    +            } finally {
    +                if (deleteAll) {
    +                    LOG.warn("Failed to download basic resources for topology-id {}", _topologyId);
    +                    _fsOps.deleteIfExists(tr);
    +                    _fsOps.deleteIfExists(_stormRoot);
    +                }
    +            }
    +            return null;
    +        }
    +    }
    +    
    +    private class DownloadBaseBlobsLocal extends DownloadBaseBlobsDistributed {
    +
    +        public DownloadBaseBlobsLocal(String topologyId) throws IOException {
    +            super(topologyId);
    +        }
    +        
    +        @Override
    +        protected void downloadBaseBlobs(File tmproot) throws Exception {
    +            _fsOps.forceMkdir(tmproot);
    +            String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +            String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +            File codePath = new File(ConfigUtils.supervisorStormCodePath(tmproot.getAbsolutePath()));
    +            File confPath = new File(ConfigUtils.supervisorStormConfPath(tmproot.getAbsolutePath()));
    +            BlobStore blobStore = Utils.getNimbusBlobStore(_conf, null);
    +            try {
    +                try (OutputStream codeOutStream = _fsOps.getOutputStream(codePath)){
    +                    blobStore.readBlobTo(stormCodeKey, codeOutStream, null);
    +                }
    +                try (OutputStream confOutStream = _fsOps.getOutputStream(confPath)) {
    +                    blobStore.readBlobTo(stormConfKey, confOutStream, null);
    +                }
    +            } finally {
    +                blobStore.shutdown();
    +            }
    +
    +            ClassLoader classloader = Thread.currentThread().getContextClassLoader();
    +            String resourcesJar = AsyncLocalizer.resourcesJar();
    +            URL url = classloader.getResource(ConfigUtils.RESOURCES_SUBDIR);
    +
    +            String targetDir = tmproot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +
    +            if (resourcesJar != null) {
    +                LOG.info("Extracting resources from jar at {} to {}", resourcesJar, targetDir);
    +                Utils.extractDirFromJar(resourcesJar, ConfigUtils.RESOURCES_SUBDIR, _stormRoot);
    +            } else if (url != null) {
    +                LOG.info("Copying resources at {} to {} ", url.toString(), targetDir);
    +                if ("jar".equals(url.getProtocol())) {
    +                    JarURLConnection urlConnection = (JarURLConnection) url.openConnection();
    +                    Utils.extractDirFromJar(urlConnection.getJarFileURL().getFile(), ConfigUtils.RESOURCES_SUBDIR, _stormRoot);
    +                } else {
    +                    _fsOps.copyDirectory(new File(url.getFile()), new File(targetDir));
    +                }
    +            }
    +        }
    +    }
    +    
    +    private class DownloadBlobs implements Callable<Void> {
    +        private final String _topologyId;
    +
    +        public DownloadBlobs(String topologyId) {
    +            _topologyId = topologyId;
    +        }
    +
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    --- End diff --
    
    The ExecutorService will store the Exception in the Future associated with making the call.  It will be thrown when .get is called on the Future.  We are doing that in Slot.  So in theory we should not miss anything, but yes you are right in practice things can be missed, and it is a huge pain when a bug is introduced and Exceptions are eaten. I'll look into it.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79031670
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,328 @@
    +/**
    + * 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.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 java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +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.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this.superConf = supervisor.getConf();
    +        this.stormClusterState = supervisor.getStormClusterState();
    +        this.syncSupEventManager = supervisor.getEventManger();
    +        this.assignmentVersions = new AtomicReference<>(new HashMap<>());
    +        this.assignmentId = supervisor.getAssignmentId();
    +        this.iSuper = supervisor.getiSupervisor();
    +        this.localizer = supervisor.getAsyncLocalizer();
    +        this.host = supervisor.getHostName();
    +        this.localState = supervisor.getLocalState();
    +        this.clusterState = supervisor.getStormClusterState();
    +        this.cachedAssignments = supervisor.getCurrAssignment();
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, supervisor.getSharedContext());
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +        
    +        try {
    +            Collection<String> workers = SupervisorUtils.supervisorWorkerIds(superConf);
    +            for (Slot slot: slots.values()) {
    +                String workerId = slot.getWorkerId();
    +                if (workerId != null) {
    +                    workers.remove(workerId);
    +                }
    +            }
    +            if (!workers.isEmpty()) {
    +                supervisor.killWorkers(workers, launcher);
    +            }
    +        } catch (Exception e) {
    +            LOG.warn("Error trying to clean up old workers", e);
    +        }
    +
    +        //All the slots/assignments should be recovered now, so we can clean up anything that we don't expect to be here
    +        try {
    +            localizer.cleanupUnusedTopologies();
    +        } catch (Exception e) {
    +            LOG.warn("Error trying to clean up old topologies", e);
    +        }
    +        
    +        for (Slot slot: slots.values()) {
    +            slot.start();
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        return new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    +                            if (actions == null) {
    +                                actions = new HashSet<>();
    +                                filtered.put(port.intValue(), actions);
    +                            }
    +                            actions.add(new TopoProfileAction(topoId, req));
    +                        }
    +                    }
    +                }
    +            }
    +            
    +            for (Integer port: allPorts) {
    +                Slot slot = slots.get(port);
    +                if (slot == null) {
    +                    slot = mkSlot(port);
    +                    slots.put(port, slot);
    +                    slot.start();
    +                }
    +                slot.setNewAssignment(allAssignments.get(port));
    +                slot.addProfilerActions(filtered.get(port));
    +            }
    +            
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +    }
    +    
    +    protected Map<String, VersionedData<Assignment>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +            Map<String, VersionedData<Assignment>> localAssignmentVersion, Runnable callback) throws Exception {
    +        Map<String, VersionedData<Assignment>> updateAssignmentVersion = new HashMap<>();
    +        for (String topoId : topoIds) {
    +            Integer recordedVersion = -1;
    +            Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +            VersionedData<Assignment> locAssignment = localAssignmentVersion.get(topoId);
    +            if (locAssignment != null) {
    +                recordedVersion = locAssignment.getVersion();
    +            }
    +            if (version == null) {
    +                // ignore
    +            } else if (version == recordedVersion) {
    +                updateAssignmentVersion.put(topoId, locAssignment);
    +            } else {
    +                VersionedData<Assignment> assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                updateAssignmentVersion.put(topoId, assignmentVersion);
    +            }
    +        }
    +        return updateAssignmentVersion;
    +    }
    +    
    +    protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +        Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +        for (String stormId : stormIds) {
    +            List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +            ret.put(stormId, profileRequests);
    +        }
    +        return ret;
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readAssignments(Map<String, VersionedData<Assignment>> assignmentsSnapshot,
    +            String assignmentId, AtomicInteger retries) {
    --- End diff --
    
    This came from the original clojure code, that was then translated into java, and I just left it.  The way it works right now is that there is a sleep in between the retries in the main thread.  I wanted to fix it, but felt leaving it would be simpler.  If you want me to I will change it.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76272537
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,749 @@
    +/**
    + * 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,
    +        WATING_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 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 monitorFreqMs,
    +                ContainerLauncher containerLauncher, String host, int port,
    +                ISupervisor iSupervisor, LocalState localState) {
    +            this.localizer = localizer;
    +            this.hbTimeoutMs = hbTimeoutMs;
    +            this.firstHbTimeoutMs = firstHbTimeoutMs;
    +            this.containerLauncher = containerLauncher;
    +            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 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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WATING_FOR_WORKER_START) {
    +                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 equivilant(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;
    +                    } else 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 WATING_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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), staticState.port);
    +        }
    +        Time.sleep(1000);
    +        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(1000);
    --- End diff --
    
    same here


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78981043
  
    --- 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);
    +        
    +        //Ignore changes to scheduling while downloading the topology blobs
    +        // We don't support canceling the download through the future yet,
    +        // so to keep everything in sync, just wait
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization, staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        // We don't support canceling the download through the future yet,
    +        // so to keep everything in sync, just wait
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization, staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        //The child processes typically exit in < 1 sec.  If 2 mins later they are still around something is wrong
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for WAITING_FOR_WORKER_START state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleWaitingForWorkerStart(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb != null) {
    +            long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +            if (hbAgeMs <= staticState.hbTimeoutMs) {
    +                return dynamicState.withState(MachineState.RUNNING);
    +            }
    +        }
    +        
    +        if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            //We were rescheduled while waiting for the worker to come up
    +            LOG.warn("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAssignment, dynamicState.newAssignment);
    +            return Slot.killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffms = (Time.currentTimeMillis() - dynamicState.startTime);
    +        if (timeDiffms > staticState.firstHbTimeoutMs) {
    +            LOG.warn("SLOT {}: Container {} failed to launch in {} ms.", staticState.port, dynamicState.container, staticState.firstHbTimeoutMs);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for RUNNING state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleRunning(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            LOG.warn("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAssignment, dynamicState.newAssignment);
    +            //Scheduling changed while running...
    +            return killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        if (dynamicState.container.didMainProcessExit()) {
    +            LOG.warn("SLOT {}: main process has exited", staticState.port);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb == null) {
    +            LOG.warn("SLOT {}: HB returned as null", staticState.port);
    +            //This can happen if the supervisor crashed after launching a
    +            // worker that never came up.
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +        if (timeDiffMs > staticState.hbTimeoutMs) {
    +            LOG.warn("SLOT {}: HB is too old {} > {}", staticState.port, timeDiffMs, staticState.hbTimeoutMs);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        //The worker is up and running check for profiling requests
    +        if (!dynamicState.profileActions.isEmpty()) {
    +            HashSet<TopoProfileAction> mod = new HashSet<>(dynamicState.profileActions);
    +            HashSet<TopoProfileAction> modPending = new HashSet<>(dynamicState.pendingStopProfileActions);
    +            Iterator<TopoProfileAction> iter = mod.iterator();
    +            while (iter.hasNext()) {
    +                TopoProfileAction action = iter.next();
    +                if (!action.topoId.equals(dynamicState.currentAssignment.get_topology_id())) {
    +                    iter.remove();
    +                    LOG.warn("Dropping {} wrong topology is running", action);
    +                    //Not for this topology so skip it
    +                } else {
    +                    if (modPending.contains(action)) {
    +                        boolean isTimeForStop = Time.currentTimeMillis() > action.request.get_time_stamp();
    +                        if (isTimeForStop) {
    +                            if (dynamicState.container.runProfiling(action.request, true)) {
    +                                LOG.debug("Stopped {} action finished", action);
    +                                iter.remove();
    +                                modPending.remove(action);
    +                            } else {
    +                                LOG.warn("Stopping {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            LOG.debug("Still pending {} now: {}", action, Time.currentTimeMillis());
    +                        }
    +                    } else {
    +                        //J_PROFILE_START is not used.  When you see a J_PROFILE_STOP
    +                        // start profiling and save it away to stop when timeout happens
    +                        if (action.request.get_action() == ProfileAction.JPROFILE_STOP) {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                modPending.add(action);
    +                                LOG.debug("Started {} now: {}", action, Time.currentTimeMillis());
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                LOG.debug("Started {} action finished", action);
    +                                iter.remove();
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        }
    +                    }
    +                }
    +            }
    +            dynamicState = dynamicState.withProfileActions(mod, modPending);
    +        }
    +        Time.sleep(staticState.monitorFreqMs);
    +        return dynamicState;
    +    }
    +
    +    static DynamicState handleEmpty(DynamicState dynamicState, StaticState staticState) throws InterruptedException, IOException {
    +        if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +        }
    +        //Both assignments are null, just wait
    +        if (dynamicState.profileActions != null && !dynamicState.profileActions.isEmpty()) {
    +            //Nothing is scheduled here so throw away all of the profileActions
    +            LOG.warn("Dropping {} no topology is running", dynamicState.profileActions);
    +            dynamicState = dynamicState.withProfileActions(Collections.<TopoProfileAction> emptySet(), Collections.<TopoProfileAction> emptySet());
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +    
    +    private final AtomicReference<LocalAssignment> newAssignment = new AtomicReference<>();
    +    private final AtomicReference<Set<TopoProfileAction>> profiling =
    +            new AtomicReference<Set<TopoProfileAction>>(new HashSet<TopoProfileAction>());
    +    private final StaticState staticState;
    +    private final IStormClusterState clusterState;
    +    private volatile boolean done = false;
    +    private volatile DynamicState dynamicState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignments;
    +    
    +    public Slot(ILocalizer localizer, Map<String, Object> conf, 
    +            ContainerLauncher containerLauncher, String host,
    +            int port, LocalState localState,
    +            IStormClusterState clusterState,
    +            ISupervisor iSupervisor,
    +            AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignments) throws Exception {
    +        super("SLOT_"+port);
    +
    +        this.cachedCurrentAssignments = cachedCurrentAssignments;
    +        this.clusterState = clusterState;
    +        Map<Integer, LocalAssignment> assignments = localState.getLocalAssignmentsMap();
    +        LocalAssignment currentAssignment = null;
    +        if (assignments != null) {
    +            currentAssignment = assignments.get(port);
    +        }
    +        Container container = null;
    +        if (currentAssignment != null) { 
    +            container = containerLauncher.recoverContainer(port, currentAssignment, localState);
    +        }
    +        
    +        LocalAssignment newAssignment = currentAssignment;
    +        if (currentAssignment != null && container == null) {
    +            currentAssignment = null;
    +            //Assigned something but it is not running
    +        }
    +        
    +        dynamicState = new DynamicState(currentAssignment, container, newAssignment);
    +        staticState = new StaticState(localizer, 
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_START_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_MONITOR_FREQUENCY_SECS)) * 1000,
    +                containerLauncher,
    +                host,
    +                port,
    +                iSupervisor,
    +                localState);
    +        this.newAssignment.set(dynamicState.newAssignment);
    +        if (MachineState.RUNNING == dynamicState.state) {
    +            //We are running so we should recover the blobs.
    +            staticState.localizer.recoverRunningTopology(currentAssignment, port);
    +            saveNewAssignment(currentAssignment);
    +        }
    +        LOG.warn("SLOT {}:{} Starting in state {} - assignment {}", staticState.host, staticState.port, dynamicState.state, dynamicState.currentAssignment);
    +    }
    +    
    +    public MachineState getMachineState() {
    +        return dynamicState.state;
    +    }
    +    
    +    /**
    +     * Set a new assignment asynchronously
    +     * @param newAssignment the new assignment for this slot to run, null to run nothing
    +     */
    +    public void setNewAssignment(LocalAssignment newAssignment) {
    +        this.newAssignment.set(newAssignment);
    +    }
    +    
    +    public void addProfilerActions(Set<TopoProfileAction> actions) {
    +        if (actions != null) {
    +            while(true) {
    +                Set<TopoProfileAction> orig = profiling.get();
    +                Set<TopoProfileAction> newActions = new HashSet<>(orig);
    +                newActions.addAll(actions);
    +                if (profiling.compareAndSet(orig, newActions)) {
    +                    return;
    +                }
    +            }
    +        }
    +    }
    +    
    +    public String getWorkerId() {
    +        String workerId = null;
    +        Container c = dynamicState.container;
    +        if (c != null) {
    +            workerId = c.getWorkerId();
    +        }
    +        return workerId;
    +    }
    +    
    +    private void saveNewAssignment(LocalAssignment assignment) {
    +        synchronized(staticState.localState) {
    +            Map<Integer, LocalAssignment> assignments = staticState.localState.getLocalAssignmentsMap();
    +            if (assignments == null) {
    +                assignments = new HashMap<>();
    +            }
    +            if (assignment == null) {
    +                assignments.remove(staticState.port);
    +            } else {
    +                assignments.put(staticState.port, assignment);
    +            }
    +            staticState.localState.setLocalAssignmentsMap(assignments);
    +        }
    +        Map<Long, LocalAssignment> update = null;
    +        Map<Long, LocalAssignment> orig = null;
    +        do {
    +            Long lport = new Long(staticState.port);
    +            orig = cachedCurrentAssignments.get();
    +            update = new HashMap<>(orig);
    +            if (assignment == null) {
    +                update.remove(lport);
    +            } else {
    +                update.put(lport, assignment);
    +            }
    +        } while (!cachedCurrentAssignments.compareAndSet(orig, update));
    +    }
    +    
    +    public void run() {
    +        try {
    +            while(!done) {
    +                LocalAssignment localNewAssignment = newAssignment.get();
    +                Set<TopoProfileAction> origProfileActions = new HashSet<>(profiling.get());
    +                Set<TopoProfileAction> removed = new HashSet<>(origProfileActions);
    +                
    +                DynamicState nextState = 
    +                        stateMachineStep(dynamicState.withNewAssignment(localNewAssignment)
    +                                .withProfileActions(origProfileActions, dynamicState.pendingStopProfileActions), staticState);
    +
    +                if (LOG.isDebugEnabled() || dynamicState.state != nextState.state) {
    +                    LOG.info("STATE {} -> {}", dynamicState, nextState);
    +                }
    +                //Save the current state for recovery
    +                if (!equivalent(nextState.currentAssignment, dynamicState.currentAssignment)) {
    +                    LOG.warn("SLOT {}: Changing current assignment from {} to {}", staticState.port, dynamicState.currentAssignment, nextState.currentAssignment);
    --- End diff --
    
    Isn't this more of an INFO level type log?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76817454
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,769 @@
    +/**
    + * 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);
    +            }
    +            if (container != null) {
    +                sb.append(" container: ");
    +                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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WAITING_FOR_WORKER_START) {
    +                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 equivilant(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;
    +                    } else 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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), 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.get_topology_id(), 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);
    +        
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivilant(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +        
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for WAITING_FOR_WORKER_START state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleWaitingForWorkerStart(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb != null) {
    +            long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +            if (hbAgeMs <= staticState.hbTimeoutMs) {
    +                return dynamicState.withState(MachineState.RUNNING);
    +            }
    +        }
    +        
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            //We were rescheduled while waiting for the worker to come up
    +            return Slot.killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffms = (Time.currentTimeMillis() - dynamicState.startTime);
    +        if (timeDiffms > staticState.firstHbTimeoutMs) {
    +            LOG.warn("SLOT {}: Container {} failed to launch in {} ms.", staticState.port, dynamicState.container, staticState.firstHbTimeoutMs);
    +            dynamicState.container.kill();
    +            Time.sleep(staticState.killSleepMs);
    +            return dynamicState.withState(MachineState.KILL_AND_RELAUNCH);
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for RUNNING state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleRunning(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            LOG.warn("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAssignment, dynamicState.newAssignment);
    +            //Scheduling changed while running...
    +            return killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        if (dynamicState.container.didMainProcessExit()) {
    +            LOG.warn("SLOT {}: main process has exited", staticState.port);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb == null) {
    +            LOG.warn("SLOT {}: HB returned as null", staticState.port);
    +            //This should never happen, but to be safe
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +        if (timeDiffMs > staticState.hbTimeoutMs) {
    +            LOG.warn("SLOT {}: HB is too old {} > {}", staticState.port, timeDiffMs, staticState.hbTimeoutMs);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        //The worker is up and running check for profiling requests
    +        if (!dynamicState.profileActions.isEmpty()) {
    +            HashSet<TopoProfileAction> mod = new HashSet<>(dynamicState.profileActions);
    +            HashSet<TopoProfileAction> modPending = new HashSet<>(dynamicState.pendingStopProfileActions);
    +            Iterator<TopoProfileAction> iter = mod.iterator();
    +            while (iter.hasNext()) {
    +                TopoProfileAction action = iter.next();
    +                if (!action.topoId.equals(dynamicState.currentAssignment.get_topology_id())) {
    +                    iter.remove();
    +                    LOG.warn("Dropping {} wrong topology is running", action);
    +                    //Not for this topology so skip it
    +                } else {
    +                    if (modPending.contains(action)) {
    +                        boolean isTimeForStop = Time.currentTimeMillis() > action.request.get_time_stamp();
    +                        if (isTimeForStop) {
    +                            if (dynamicState.container.runProfiling(action.request, true)) {
    +                                LOG.debug("Stopped {} action finished", action);
    +                                iter.remove();
    +                                modPending.remove(action);
    +                            } else {
    +                                LOG.warn("Stopping {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            LOG.debug("Still pending {} now: {}", action, Time.currentTimeMillis());
    +                        }
    +                    } else {
    +                        //J_PROFILE_START is not used.  When you see a J_PROFILE_STOP
    +                        // start profiling and save it away to stop when timeout happens
    +                        if (action.request.get_action() == ProfileAction.JPROFILE_STOP) {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                modPending.add(action);
    +                                LOG.debug("Started {} now: {}", action, Time.currentTimeMillis());
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                LOG.debug("Started {} action finished", action);
    +                                iter.remove();
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        }
    +                    }
    +                }
    +            }
    +            dynamicState = dynamicState.withProfileActions(mod, modPending);
    +        }
    +        Time.sleep(staticState.monitorFreqMs);
    +        return dynamicState;
    +    }
    +
    +    static DynamicState handleEmpty(DynamicState dynamicState, StaticState staticState) throws InterruptedException {
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +        }
    +        //Both assignments are null, just wait
    +        if (dynamicState.profileActions != null && !dynamicState.profileActions.isEmpty()) {
    +            //Nothing is scheduled here so throw away all of the profileActions
    +            LOG.warn("Dropping {} no topology is running", dynamicState.profileActions);
    +            dynamicState = dynamicState.withProfileActions(Collections.<TopoProfileAction> emptySet(), Collections.<TopoProfileAction> emptySet());
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +    
    +    private final AtomicReference<LocalAssignment> newAssignment = new AtomicReference<>();
    +    private final AtomicReference<Set<TopoProfileAction>> profiling =
    +            new AtomicReference<Set<TopoProfileAction>>(new HashSet<TopoProfileAction>());
    +    private final StaticState staticState;
    +    private final IStormClusterState clusterState;
    +    private volatile boolean done = false;
    +    private volatile DynamicState dynamicState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignmants;
    +    
    +    public Slot(ILocalizer localizer, Map<String, Object> conf, 
    +            ContainerLauncher containerLauncher, String host,
    +            int port, LocalState localState,
    +            IStormClusterState clusterState,
    +            ISupervisor iSupervisor,
    +            AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignmants) throws Exception {
    +        super("SLOT_"+port);
    +
    +        this.cachedCurrentAssignmants = cachedCurrentAssignmants;
    +        this.clusterState = clusterState;
    +        Map<Integer, LocalAssignment> assignments = localState.getLocalAssignmentsMap();
    +        LocalAssignment currentAssignment = null;
    +        if (assignments != null) {
    +            currentAssignment = assignments.get(port);
    +        }
    +        Container container = null;
    +        if (currentAssignment != null) { 
    +            container = containerLauncher.recoverContainer(port, currentAssignment, localState);
    +        }
    +        
    +        LocalAssignment newAssignment = currentAssignment;
    +        if (currentAssignment != null && container == null) {
    +            currentAssignment = null;
    +            //Assigned something but it is not running
    +        }
    +        
    +        dynamicState = new DynamicState(currentAssignment, container, newAssignment);
    +        staticState = new StaticState(localizer, 
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_START_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_MONITOR_FREQUENCY_SECS)) * 1000,
    +                containerLauncher,
    +                host,
    +                port,
    +                iSupervisor,
    +                localState);
    +        this.newAssignment.set(dynamicState.newAssignment);
    +        if (MachineState.RUNNING == dynamicState.state) {
    +            //We are running so we should recover the blobs.
    +            staticState.localizer.recoverRunningTopology(currentAssignment.get_topology_id(), port);
    +        }
    +        LOG.warn("SLOT {}:{} Starting in state {} - assignment {}", staticState.host, staticState.port, dynamicState.state, dynamicState.currentAssignment);
    +    }
    +    
    +    public MachineState getMachineState() {
    +        return dynamicState.state;
    +    }
    +    
    +    /**
    +     * Set a new assignment asynchronously
    +     * @param newAssignment the new assignment for this slot to run, null to run nothing
    +     */
    +    public void setNewAssignment(LocalAssignment newAssignment) {
    +        this.newAssignment.set(newAssignment);
    +    }
    +    
    +    public void addProfilerActions(Set<TopoProfileAction> actions) {
    +        if (actions != null) {
    +            while(true) {
    +                Set<TopoProfileAction> orig = profiling.get();
    +                Set<TopoProfileAction> newActions = new HashSet<>(orig);
    +                newActions.addAll(actions);
    +                if (profiling.compareAndSet(orig, newActions)) {
    +                    return;
    +                }
    +            }
    +        }
    +    }
    +    
    +    public void run() {
    +        try {
    +            while(!done) {
    +                LocalAssignment localNewAssignment = newAssignment.get();
    +                Set<TopoProfileAction> origProfileActions = new HashSet<>(profiling.get());
    +                Set<TopoProfileAction> removed = new HashSet<>(origProfileActions);
    +                
    +                DynamicState nextState = 
    +                        stateMachineStep(dynamicState.withNewAssignment(localNewAssignment)
    +                                .withProfileActions(origProfileActions, dynamicState.pendingStopProfileActions), staticState);
    +
    +                if (LOG.isDebugEnabled() || dynamicState.state != nextState.state) {
    +                    LOG.info("STATE {} -> {}", dynamicState, nextState);
    +                }
    +                //Save the current state for recovery
    +                if (!equivilant(nextState.currentAssignment, dynamicState.currentAssignment)) {
    +                    LOG.warn("SLOT {}: Changing current assignment from {} to {}", staticState.port, dynamicState.currentAssignment, nextState.currentAssignment);
    +                    synchronized(staticState.localState) {
    +                        Map<Integer, LocalAssignment> assignments = staticState.localState.getLocalAssignmentsMap();
    +                        if (assignments == null) {
    +                            assignments = new HashMap<>();
    +                        }
    +                        if (nextState.currentAssignment == null) {
    +                            assignments.remove(staticState.port);
    +                        } else {
    +                            assignments.put(staticState.port, nextState.currentAssignment);
    +                        }
    +                        staticState.localState.setLocalAssignmentsMap(assignments);
    +                    }
    +                    Map<Long, LocalAssignment> update = null;
    +                    Map<Long, LocalAssignment> orig = null;
    +                    do {
    +                        Long lport = new Long(staticState.port);
    +                        orig = cachedCurrentAssignmants.get();
    --- End diff --
    
    cachedCurrentAssignmants -> cachedCurrentAssignments


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76162679
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,417 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.AdvancedFSOps;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill the processes in this container nicely.
    +     * kill -15 equivalent
    +     * @throws IOException on any error
    +     */
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Kill the processes in this container violently.
    +     * kill -9 equivalent
    +     * @throws IOException on any error
    +     */
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq"+pid);
    +            pb.redirectError(Redirect.INHERIT);
    +            Process p = pb.start();
    +            try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +                //TODO would be really nice to parse out the user
    +                if (in.readLine() != null) {
    +                    return true;
    +                }
    +            }
    +            return false;
    +        }
    +        //Not on Windows assume POSIX
    +        ProcessBuilder pb = new ProcessBuilder("ps", "-o", "user", "-p", String.valueOf(pid));
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            String first = in.readLine();
    +            assert("USER".equals(first));
    +            String processUser;
    +            while ((processUser = in.readLine()) != null) {
    +                if (user.equals(processUser)) {
    +                    return true;
    +                } else {
    +                    LOG.info("Found {} running as {}, but expected it to be {}", pid, processUser, user);
    +                }
    +            }
    +        }
    +        return false;
    +    }
    +    
    +    /**
    +     * @return true if all of the processes are dead, else false
    +     * @throws IOException on any error
    +     */
    +    public boolean areAllProcessesDead() throws IOException {
    +        Set<Long> pids = getAllPids();
    +        String user = getWorkerUser();
    +        
    +        boolean allDead = true;
    +        for (Long pid: pids) {
    +            if (!isProcessAlive(pid, user)) {
    +                LOG.warn("{}: PID {} is dead", _workerId, pid);
    +            } else {
    +                allDead = false;
    +                break;
    +            }
    +        }
    +        return allDead;
    +    }
    +
    +    /**
    +     * Clean up the container. It is not coming back.
    +     * by default do the same thing as when restarting.
    +     * @throws IOException on any error
    +     */
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +    }
    +
    +    /**
    +     * Setup the container to run.  By default this creates the needed directories/links in the
    +     * local file system
    +     * PREREQUISITE: All needed blobs and topology, jars/configs have been downloaded and
    +     * placed in the appropriate locations
    +     * @throws IOException on any error
    +     */
    +    public void setup() throws IOException {
    +        final String topologyId = _assignment.get_topology_id();
    +        if (!SupervisorUtils.doRequiredTopoFilesExist(_conf, topologyId)) {
    +            LOG.info("Missing topology storm code, so can't launch  worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                    _supervisorId, _port, _workerId);
    +            throw new IllegalStateException("Not all needed files are here!!!!");
    +        }
    +        String pidsPath = ConfigUtils.workerPidsRoot(_conf, _workerId);
    +        String hbPath = ConfigUtils.workerHeartbeatsRoot(_conf, _workerId);
    +    
    +        FileUtils.forceMkdir(new File(pidsPath));
    +        FileUtils.forceMkdir(new File(ConfigUtils.workerTmpRoot(_conf, _workerId)));
    +        FileUtils.forceMkdir(new File(hbPath));
    +    
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        String user = (String) topologyConf.get(Config.TOPOLOGY_SUBMITTER_USER);
    +        writeLogMetadata(topologyConf, user, topologyId);
    +        ConfigUtils.setWorkerUserWSE(_conf, _workerId, user);
    +        createArtifactsLink(topologyId);
    +    
    +        createBlobstoreLinks(topologyId);
    +    }
    +    
    +    /**
    +     * Write out the file used by the log viewer to allow/reject log access
    +     * @param topologyConf the config for the topology
    +     * @param user the user this is going to run as
    +     * @param topologyId the id of the topology
    +     * @throws IOException on any error
    +     */
    +    @SuppressWarnings("unchecked")
    +    protected void writeLogMetadata(Map<String, Object> topologyConf, String user, String topologyId) throws IOException {
    +        Map<String, Object> data = new HashMap<>();
    +        data.put(Config.TOPOLOGY_SUBMITTER_USER, user);
    +        data.put("worker-id", _workerId);
    +
    +        Set<String> logsGroups = new HashSet<>();
    +        //for supervisor-test
    +        if (topologyConf.get(Config.LOGS_GROUPS) != null) {
    +            List<String> groups = (List<String>) topologyConf.get(Config.LOGS_GROUPS);
    +            for (String group : groups){
    +                logsGroups.add(group);
    +            }
    +        }
    +        if (topologyConf.get(Config.TOPOLOGY_GROUPS) != null) {
    +            List<String> topGroups = (List<String>) topologyConf.get(Config.TOPOLOGY_GROUPS);
    +            logsGroups.addAll(topGroups);
    +        }
    +        data.put(Config.LOGS_GROUPS, logsGroups.toArray());
    +
    +        Set<String> logsUsers = new HashSet<>();
    +        if (topologyConf.get(Config.LOGS_USERS) != null) {
    +            List<String> logUsers = (List<String>) topologyConf.get(Config.LOGS_USERS);
    +            for (String logUser : logUsers){
    +                logsUsers.add(logUser);
    +            }
    +        }
    +        if (topologyConf.get(Config.TOPOLOGY_USERS) != null) {
    +            List<String> topUsers = (List<String>) topologyConf.get(Config.TOPOLOGY_USERS);
    +            for (String logUser : topUsers){
    +                logsUsers.add(logUser);
    +            }
    +        }
    +        data.put(Config.LOGS_USERS, logsUsers.toArray());
    +
    +        File file = ConfigUtils.getLogMetaDataFile(_conf, topologyId, _port);
    +        File parent = file.getParentFile();
    +        if (!Utils.checkFileExists(parent)) {
    +            FileUtils.forceMkdir(file.getParentFile());
    +            _ops.setupStormCodeDir(ConfigUtils.readSupervisorStormConf(_conf, topologyId), file.getParentFile().getCanonicalPath());
    +        }
    +        Yaml yaml = new Yaml();
    +        try (FileWriter writer = new FileWriter(file)) {
    +            yaml.dump(data, writer);
    +        }
    +    }
    +    
    +    /**
    +     * Create symlink from the containers directory/artifacts to the artifacts directory
    +     * @param topologyId the id of the topology this is for
    +     * @throws IOException on any error
    +     */
    +    protected void createArtifactsLink(String topologyId) throws IOException {
    +        String workerDir = ConfigUtils.workerRoot(_conf, _workerId);
    +        String topoDir = ConfigUtils.workerArtifactsRoot(_conf, topologyId);
    +        if (Utils.checkFileExists(workerDir)) {
    +            LOG.debug("Creating symlinks for worker-id: {} topology-id: {} to its port artifacts directory", _workerId, topologyId);
    +            Utils.createSymlink(workerDir, topoDir, "artifacts", String.valueOf(_port));
    +        }
    --- End diff --
    
    Failure here seems like something we might want to at least log.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76875102
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,769 @@
    +/**
    + * 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);
    +            }
    +            if (container != null) {
    +                sb.append(" container: ");
    +                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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WAITING_FOR_WORKER_START) {
    +                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 equivilant(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;
    +                    } else 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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), 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.get_topology_id(), 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);
    +        
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivilant(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +        
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for WAITING_FOR_WORKER_START state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleWaitingForWorkerStart(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb != null) {
    +            long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +            if (hbAgeMs <= staticState.hbTimeoutMs) {
    +                return dynamicState.withState(MachineState.RUNNING);
    +            }
    +        }
    +        
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            //We were rescheduled while waiting for the worker to come up
    +            return Slot.killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffms = (Time.currentTimeMillis() - dynamicState.startTime);
    +        if (timeDiffms > staticState.firstHbTimeoutMs) {
    +            LOG.warn("SLOT {}: Container {} failed to launch in {} ms.", staticState.port, dynamicState.container, staticState.firstHbTimeoutMs);
    +            dynamicState.container.kill();
    +            Time.sleep(staticState.killSleepMs);
    +            return dynamicState.withState(MachineState.KILL_AND_RELAUNCH);
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for RUNNING state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleRunning(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            LOG.warn("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAssignment, dynamicState.newAssignment);
    +            //Scheduling changed while running...
    +            return killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        if (dynamicState.container.didMainProcessExit()) {
    +            LOG.warn("SLOT {}: main process has exited", staticState.port);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb == null) {
    +            LOG.warn("SLOT {}: HB returned as null", staticState.port);
    +            //This should never happen, but to be safe
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +        if (timeDiffMs > staticState.hbTimeoutMs) {
    +            LOG.warn("SLOT {}: HB is too old {} > {}", staticState.port, timeDiffMs, staticState.hbTimeoutMs);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        //The worker is up and running check for profiling requests
    +        if (!dynamicState.profileActions.isEmpty()) {
    +            HashSet<TopoProfileAction> mod = new HashSet<>(dynamicState.profileActions);
    +            HashSet<TopoProfileAction> modPending = new HashSet<>(dynamicState.pendingStopProfileActions);
    +            Iterator<TopoProfileAction> iter = mod.iterator();
    +            while (iter.hasNext()) {
    +                TopoProfileAction action = iter.next();
    +                if (!action.topoId.equals(dynamicState.currentAssignment.get_topology_id())) {
    +                    iter.remove();
    +                    LOG.warn("Dropping {} wrong topology is running", action);
    +                    //Not for this topology so skip it
    +                } else {
    +                    if (modPending.contains(action)) {
    +                        boolean isTimeForStop = Time.currentTimeMillis() > action.request.get_time_stamp();
    +                        if (isTimeForStop) {
    +                            if (dynamicState.container.runProfiling(action.request, true)) {
    +                                LOG.debug("Stopped {} action finished", action);
    +                                iter.remove();
    +                                modPending.remove(action);
    +                            } else {
    +                                LOG.warn("Stopping {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            LOG.debug("Still pending {} now: {}", action, Time.currentTimeMillis());
    +                        }
    +                    } else {
    +                        //J_PROFILE_START is not used.  When you see a J_PROFILE_STOP
    +                        // start profiling and save it away to stop when timeout happens
    +                        if (action.request.get_action() == ProfileAction.JPROFILE_STOP) {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                modPending.add(action);
    +                                LOG.debug("Started {} now: {}", action, Time.currentTimeMillis());
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                LOG.debug("Started {} action finished", action);
    +                                iter.remove();
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        }
    +                    }
    +                }
    +            }
    +            dynamicState = dynamicState.withProfileActions(mod, modPending);
    +        }
    +        Time.sleep(staticState.monitorFreqMs);
    +        return dynamicState;
    +    }
    +
    +    static DynamicState handleEmpty(DynamicState dynamicState, StaticState staticState) throws InterruptedException {
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +        }
    +        //Both assignments are null, just wait
    +        if (dynamicState.profileActions != null && !dynamicState.profileActions.isEmpty()) {
    +            //Nothing is scheduled here so throw away all of the profileActions
    +            LOG.warn("Dropping {} no topology is running", dynamicState.profileActions);
    +            dynamicState = dynamicState.withProfileActions(Collections.<TopoProfileAction> emptySet(), Collections.<TopoProfileAction> emptySet());
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +    
    +    private final AtomicReference<LocalAssignment> newAssignment = new AtomicReference<>();
    +    private final AtomicReference<Set<TopoProfileAction>> profiling =
    +            new AtomicReference<Set<TopoProfileAction>>(new HashSet<TopoProfileAction>());
    +    private final StaticState staticState;
    +    private final IStormClusterState clusterState;
    +    private volatile boolean done = false;
    +    private volatile DynamicState dynamicState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignmants;
    +    
    +    public Slot(ILocalizer localizer, Map<String, Object> conf, 
    +            ContainerLauncher containerLauncher, String host,
    +            int port, LocalState localState,
    +            IStormClusterState clusterState,
    +            ISupervisor iSupervisor,
    +            AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignmants) throws Exception {
    +        super("SLOT_"+port);
    +
    +        this.cachedCurrentAssignmants = cachedCurrentAssignmants;
    +        this.clusterState = clusterState;
    +        Map<Integer, LocalAssignment> assignments = localState.getLocalAssignmentsMap();
    +        LocalAssignment currentAssignment = null;
    +        if (assignments != null) {
    +            currentAssignment = assignments.get(port);
    +        }
    +        Container container = null;
    +        if (currentAssignment != null) { 
    +            container = containerLauncher.recoverContainer(port, currentAssignment, localState);
    +        }
    +        
    +        LocalAssignment newAssignment = currentAssignment;
    +        if (currentAssignment != null && container == null) {
    +            currentAssignment = null;
    +            //Assigned something but it is not running
    +        }
    +        
    +        dynamicState = new DynamicState(currentAssignment, container, newAssignment);
    +        staticState = new StaticState(localizer, 
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_START_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_MONITOR_FREQUENCY_SECS)) * 1000,
    +                containerLauncher,
    +                host,
    +                port,
    +                iSupervisor,
    +                localState);
    +        this.newAssignment.set(dynamicState.newAssignment);
    +        if (MachineState.RUNNING == dynamicState.state) {
    +            //We are running so we should recover the blobs.
    +            staticState.localizer.recoverRunningTopology(currentAssignment.get_topology_id(), port);
    +        }
    +        LOG.warn("SLOT {}:{} Starting in state {} - assignment {}", staticState.host, staticState.port, dynamicState.state, dynamicState.currentAssignment);
    +    }
    +    
    +    public MachineState getMachineState() {
    +        return dynamicState.state;
    +    }
    +    
    +    /**
    +     * Set a new assignment asynchronously
    +     * @param newAssignment the new assignment for this slot to run, null to run nothing
    +     */
    +    public void setNewAssignment(LocalAssignment newAssignment) {
    +        this.newAssignment.set(newAssignment);
    +    }
    +    
    +    public void addProfilerActions(Set<TopoProfileAction> actions) {
    +        if (actions != null) {
    +            while(true) {
    +                Set<TopoProfileAction> orig = profiling.get();
    +                Set<TopoProfileAction> newActions = new HashSet<>(orig);
    +                newActions.addAll(actions);
    +                if (profiling.compareAndSet(orig, newActions)) {
    +                    return;
    +                }
    +            }
    +        }
    +    }
    +    
    +    public void run() {
    +        try {
    +            while(!done) {
    +                LocalAssignment localNewAssignment = newAssignment.get();
    +                Set<TopoProfileAction> origProfileActions = new HashSet<>(profiling.get());
    +                Set<TopoProfileAction> removed = new HashSet<>(origProfileActions);
    +                
    +                DynamicState nextState = 
    +                        stateMachineStep(dynamicState.withNewAssignment(localNewAssignment)
    +                                .withProfileActions(origProfileActions, dynamicState.pendingStopProfileActions), staticState);
    +
    +                if (LOG.isDebugEnabled() || dynamicState.state != nextState.state) {
    +                    LOG.info("STATE {} -> {}", dynamicState, nextState);
    +                }
    +                //Save the current state for recovery
    +                if (!equivilant(nextState.currentAssignment, dynamicState.currentAssignment)) {
    +                    LOG.warn("SLOT {}: Changing current assignment from {} to {}", staticState.port, dynamicState.currentAssignment, nextState.currentAssignment);
    +                    synchronized(staticState.localState) {
    +                        Map<Integer, LocalAssignment> assignments = staticState.localState.getLocalAssignmentsMap();
    +                        if (assignments == null) {
    +                            assignments = new HashMap<>();
    +                        }
    +                        if (nextState.currentAssignment == null) {
    +                            assignments.remove(staticState.port);
    +                        } else {
    +                            assignments.put(staticState.port, nextState.currentAssignment);
    +                        }
    +                        staticState.localState.setLocalAssignmentsMap(assignments);
    +                    }
    +                    Map<Long, LocalAssignment> update = null;
    +                    Map<Long, LocalAssignment> orig = null;
    +                    do {
    +                        Long lport = new Long(staticState.port);
    +                        orig = cachedCurrentAssignmants.get();
    +                        update = new HashMap<>(orig);
    +                        if (nextState.currentAssignment == null) {
    +                            update.remove(lport);
    +                        } else {
    +                            update.put(lport, nextState.currentAssignment);
    +                        }
    +                    } while (!cachedCurrentAssignmants.compareAndSet(orig, update));
    +                }
    +                
    +                // clean up the profiler actions that are not being processed
    +                removed.removeAll(dynamicState.profileActions);
    +                removed.removeAll(dynamicState.pendingStopProfileActions);
    +                for (TopoProfileAction action: removed) {
    +                    try {
    +                        clusterState.deleteTopologyProfileRequests(action.topoId, action.request);
    +                    } catch (Exception e) {
    +                        LOG.error("Error trying to remove profiling request, it will be retried", e);
    +                    }
    +                }
    +                Set<TopoProfileAction> orig, copy;
    +                do {
    +                    orig = profiling.get();
    +                    copy = new HashSet<>(orig);
    +                    copy.removeAll(removed);
    +                } while (!profiling.compareAndSet(orig, copy));
    +                dynamicState = nextState;
    +                newAssignment.compareAndSet(localNewAssignment, dynamicState.newAssignment);
    --- End diff --
    
    This is where all of the threading is hidden away from the state machine.  There are only a few APIs that interact with slot asynchronously.  setNewAssignment is one of them, and profiling requests is the other.  To hide this from the state machine we take a snapshot of the new assignment before running the state machine, and then before we overwrite the result we check to see if it didn't change in the background. 


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77418561
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,420 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + * TODO once we have replaced the original supervisor merge this with
    + * Localizer and optimize them
    + */
    +public class AsyncLocalizer implements ILocalizer, Shutdownable {
    +    /**
    +     * A future that has already completed.
    +     */
    +    private static class AllDoneFuture implements Future<Void> {
    +
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isCancelled() {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isDone() {
    +            return true;
    +        }
    +
    +        @Override
    +        public Void get() {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void get(long timeout, TimeUnit unit) {
    +            return null;
    +        }
    +
    +    }
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizer.class);
    +
    +    private final Localizer _localizer;
    +    private final ExecutorService _execService;
    +    private final boolean _isLocalMode;
    +    private final Map<String, Object> _conf;
    +    private final Map<String, LocalDownloadedResource> _basicPending;
    +    private final Map<String, LocalDownloadedResource> _blobPending;
    +    private final AdvancedFSOps _fsOps;
    +
    +    private class DownloadBaseBlobsDistributed implements Callable<Void> {
    +        private final String _topologyId;
    +        
    +        public DownloadBaseBlobsDistributed(String topologyId) {
    +            this._topologyId = topologyId;
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            File sr = new File(stormroot);
    +            if (sr.exists()) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    Utils.forceDelete(stormroot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    --- End diff --
    
    Let's make a helper method for this `if`, since we do the same thing for the DownloadBaseBlobsLocal.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76462779
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,202 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.nio.file.FileSystems;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    +    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
    +    
    +    /**
    +     * Factory to create a new AdvancedFSOps
    +     * @param conf the configuration of the process
    +     * @return the appropriate instance of the class for this config and environment.
    +     */
    +    public static AdvancedFSOps mk(Map<String, Object> conf) {
    +        if (Utils.isOnWindows()) {
    +            return new AdvancedWindowsFSOps(conf);
    +        }
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new AdvancedRunAsUserFSOps(conf);
    +        }
    +        return new AdvancedFSOps();
    +    }
    +    
    +    private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
    +        private final Map<String, Object> _conf;
    +        
    +        public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
    +            if (Utils.isOnWindows()) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    --- End diff --
    
    UnsupportedOperationException


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77356337
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    --- End diff --
    
    minor: We may want to use `this.` as we do above for private vars just to be consistent in the file. Not a big deal though.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78963034
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,328 @@
    +/**
    + * 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.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 java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +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.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this.superConf = supervisor.getConf();
    +        this.stormClusterState = supervisor.getStormClusterState();
    +        this.syncSupEventManager = supervisor.getEventManger();
    +        this.assignmentVersions = new AtomicReference<>(new HashMap<>());
    +        this.assignmentId = supervisor.getAssignmentId();
    +        this.iSuper = supervisor.getiSupervisor();
    +        this.localizer = supervisor.getAsyncLocalizer();
    +        this.host = supervisor.getHostName();
    +        this.localState = supervisor.getLocalState();
    +        this.clusterState = supervisor.getStormClusterState();
    +        this.cachedAssignments = supervisor.getCurrAssignment();
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, supervisor.getSharedContext());
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +        
    +        try {
    +            Collection<String> workers = SupervisorUtils.supervisorWorkerIds(superConf);
    +            for (Slot slot: slots.values()) {
    +                String workerId = slot.getWorkerId();
    +                if (workerId != null) {
    +                    workers.remove(workerId);
    +                }
    +            }
    +            if (!workers.isEmpty()) {
    +                supervisor.killWorkers(workers, launcher);
    +            }
    +        } catch (Exception e) {
    +            LOG.warn("Error trying to clean up old workers", e);
    +        }
    +
    +        //All the slots/assignments should be recovered now, so we can clean up anything that we don't expect to be here
    +        try {
    +            localizer.cleanupUnusedTopologies();
    +        } catch (Exception e) {
    +            LOG.warn("Error trying to clean up old topologies", e);
    +        }
    +        
    +        for (Slot slot: slots.values()) {
    +            slot.start();
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        return new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    +                            if (actions == null) {
    +                                actions = new HashSet<>();
    +                                filtered.put(port.intValue(), actions);
    +                            }
    +                            actions.add(new TopoProfileAction(topoId, req));
    +                        }
    +                    }
    +                }
    +            }
    +            
    +            for (Integer port: allPorts) {
    +                Slot slot = slots.get(port);
    +                if (slot == null) {
    +                    slot = mkSlot(port);
    +                    slots.put(port, slot);
    +                    slot.start();
    +                }
    +                slot.setNewAssignment(allAssignments.get(port));
    +                slot.addProfilerActions(filtered.get(port));
    +            }
    +            
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +    }
    +    
    +    protected Map<String, VersionedData<Assignment>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +            Map<String, VersionedData<Assignment>> localAssignmentVersion, Runnable callback) throws Exception {
    +        Map<String, VersionedData<Assignment>> updateAssignmentVersion = new HashMap<>();
    +        for (String topoId : topoIds) {
    +            Integer recordedVersion = -1;
    +            Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +            VersionedData<Assignment> locAssignment = localAssignmentVersion.get(topoId);
    +            if (locAssignment != null) {
    +                recordedVersion = locAssignment.getVersion();
    +            }
    +            if (version == null) {
    +                // ignore
    +            } else if (version == recordedVersion) {
    +                updateAssignmentVersion.put(topoId, locAssignment);
    +            } else {
    +                VersionedData<Assignment> assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                updateAssignmentVersion.put(topoId, assignmentVersion);
    +            }
    +        }
    +        return updateAssignmentVersion;
    +    }
    +    
    +    protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +        Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +        for (String stormId : stormIds) {
    +            List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +            ret.put(stormId, profileRequests);
    +        }
    +        return ret;
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readAssignments(Map<String, VersionedData<Assignment>> assignmentsSnapshot,
    +            String assignmentId, AtomicInteger retries) {
    --- End diff --
    
    Why is the readRetry field passed as a parameter here?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78956720
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected final String _stormHome = System.getProperty("storm.home");
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +    
    +    /**
    +     * Create a new BasicContainer
    +     * @param type the type of container being made.
    +     * @param conf the supervisor config
    +     * @param supervisorId the ID of the supervisor this is a part of.
    +     * @param port the port the container is on.  Should be <= 0 if only a partial recovery
    +     * @param assignment the assignment for this container. Should be null if only a partial recovery.
    +     * @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used.
    +     * @param localState the local state of the supervisor.  May be null if partial recovery
    +     * @param workerId the id of the worker to use.  Must not be null if doing a partial recovery.
    +     * @param ops file system operations (mostly for testing) if null a new one is made
    +     * @param topoConf the config of the topology (mostly for testing) if null 
    +     * and not a partial recovery the real conf is read.
    +     * @param profileCmd the command to use when profiling (used for testing)
    +     */
    +    public BasicContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int port,
    +            LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
    +            LocalState localState, String workerId, Map<String, Object> topoConf, 
    +            AdvancedFSOps ops, String profileCmd) throws IOException {
    +        super(type, conf, supervisorId, port, assignment, resourceIsolationManager, workerId, topoConf, ops);
    +        assert(localState != null);
    +        _localState = localState;
    +
    +        if (type.isRecovery() && !type.isOnlyKillable()) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                LOG.info("Recovered Worker {}", wid);
    +                _workerId = wid;
    +            }
    +        } else if (_workerId == null){
    +            createNewWorkerId();
    +        }
    +
    +        if (profileCmd == null) {
    +            profileCmd = _stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                    + conf.get(Config.WORKER_PROFILER_COMMAND);
    +        }
    +        _profileCmd = profileCmd;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        _type.assertFull();
    +        assert(_workerId == null);
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            removeWorkersOn(workerToPort, _port);
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +            LOG.info("Created Worker ID {}", _workerId);
    +        }
    +    }
    +
    +    private static void removeWorkersOn(Map<String, Integer> workerToPort, int _port) {
    +        for (Iterator<Entry<String, Integer>> i = workerToPort.entrySet().iterator(); i.hasNext();) {
    +            Entry<String, Integer> found = i.next();
    +            if (_port == found.getValue().intValue()) {
    +                LOG.warn("Deleting worker {} from state", found.getKey());
    +                i.remove();
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUpForRestart() throws IOException {
    +        String origWorkerId = _workerId;
    +        super.cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(origWorkerId);
    +            removeWorkersOn(workersToPort, _port);
    +            _localState.setApprovedWorkers(workersToPort);
    +            LOG.info("Removed Worker ID {}", origWorkerId);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        _type.assertFull();
    +        //We are launching it now...
    +        _type = ContainerType.LAUNCH;
    +        createNewWorkerId();
    +        setup();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        _type.assertFull();
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        _type.assertFull();
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = _ops.slurpString(new File(str)).trim();
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        if (command.size() > 0) {
    +            return runProfilingCommand(command, env, logPrefix, targetFile);
    +        }
    +        LOG.warn("PROFILING REQUEST NOT SUPPORTED {} IGNORED...", request);
    +        return true;
    +    }
    +
    +    /**
    +     * Get the command to run when doing profiling
    +     * @param action the profiling action to perform
    +     * @param stop if this is meant to stop the profiling or start it
    +     * @param workerPid the PID of the process to profile
    +     * @param targetDir the current working directory of the worker process
    +     * @return the command to run for profiling.
    +     */
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        switch(action) {
    +            case JMAP_DUMP:
    +                return jmapDumpCmd(workerPid, targetDir);
    +            case JSTACK_DUMP:
    +                return jstackDumpCmd(workerPid, targetDir);
    +            case JPROFILE_DUMP:
    +                return jprofileDump(workerPid, targetDir);
    +            case JVM_RESTART:
    +                return jprofileJvmRestart(workerPid);
    +            case JPROFILE_STOP:
    +                if (stop) {
    +                    return jprofileStop(workerPid, targetDir);
    +                }
    +                return jprofileStart(workerPid);
    +            default:
    +                return Lists.newArrayList();
    +        }
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +
    +    /**
    +     * Compute the java.library.path that should be used for the worker.
    +     * This helps it to load JNI libraries that are packaged in the uber jar.
    +     * @param stormRoot the root directory of the worker process
    +     * @param conf the config for the supervisor.
    +     * @return the java.library.path/LD_LIBRARY_PATH to use so native libraries load correctly.
    +     */
    +    protected String javaLibraryPath(String stormRoot, Map<String, Object> conf) {
    +        String resourceRoot = stormRoot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +        String os = System.getProperty("os.name").replaceAll("\\s+", "_");
    +        String arch = System.getProperty("os.arch");
    +        String archResourceRoot = resourceRoot + Utils.FILE_PATH_SEPARATOR + os + "-" + arch;
    +        String ret = CPJ.join(archResourceRoot, resourceRoot,
    +                conf.get(Config.JAVA_LIBRARY_PATH));
    +        return ret;
    +    }
    +
    +    /**
    +     * Returns a collection of jar file names found under the given directory.
    +     * @param dir the directory to search
    +     * @return the jar file names
    +     */
    +    protected List<String> getFullJars(File dir) {
    +        File[] files = dir.listFiles(jarFilter);
    +
    +        if (files == null) {
    +            return Collections.emptyList();
    +        }
    +
    +        return Arrays.stream(files).map(f -> f.getAbsolutePath())
    +                .collect(Collectors.toList());
    +    }
    +    
    +    protected List<String> frameworkClasspath() {
    +        File stormLibDir = new File(_stormHome, "lib");
    +        String stormConfDir =
    +                System.getenv("STORM_CONF_DIR") != null ?
    +                System.getenv("STORM_CONF_DIR") :
    +                new File(_stormHome, "conf").getAbsolutePath();
    +        File stormExtlibDir = new File(_stormHome, "extlib");
    +        String extcp = System.getenv("STORM_EXT_CLASSPATH");
    +        List<String> pathElements = new LinkedList<>();
    +        pathElements.addAll(getFullJars(stormLibDir));
    +        pathElements.addAll(getFullJars(stormExtlibDir));
    +        pathElements.add(extcp);
    +        pathElements.add(stormConfDir);
    +
    +        return pathElements;
    +    }
    +    
    +    @SuppressWarnings("unchecked")
    +    private List<String> asStringList(Object o) {
    +        if (o instanceof String) {
    +            return Arrays.asList((String)o);
    +        } else if (o instanceof List) {
    +            return (List<String>)o;
    +        }
    +        return Collections.EMPTY_LIST;
    +    }
    +    
    +    /**
    +     * Compute the classpath for the worker process
    +     * @param stormJar the topology jar
    +     * @param dependencyLocations any dependencies from the topology
    +     * @return the full classpath
    +     */
    +    protected String getWorkerClassPath(String stormJar, List<String> dependencyLocations) {
    +        List<String> workercp = new ArrayList<>();
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH_BEGINNING)));
    +        workercp.addAll(frameworkClasspath());
    +        workercp.add(stormJar);
    +        workercp.addAll(dependencyLocations);
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH)));
    +        return CPJ.join(workercp);
    +    }
    +
    +    private String substituteChildOptsInternal(String string, int memOnheap) {
    +        if (StringUtils.isNotBlank(string)) {
    +            String p = String.valueOf(_port);
    +            string = string.replace("%ID%", p);
    +            string = string.replace("%WORKER-ID%", _workerId);
    +            string = string.replace("%TOPOLOGY-ID%", _topologyId);
    +            string = string.replace("%WORKER-PORT%", p);
    +            if (memOnheap > 0) {
    +                string = string.replace("%HEAP-MEM%", String.valueOf(memOnheap));
    +            }
    +        }
    +        return string;
    +    }
    +    
    +    protected List<String> substituteChildopts(Object value) {
    +        return substituteChildopts(value, -1);
    +    }
    +
    +    protected List<String> substituteChildopts(Object value, int memOnheap) {
    +        List<String> rets = new ArrayList<>();
    +        if (value instanceof String) {
    +            String string = substituteChildOptsInternal((String) value, memOnheap);
    +            if (StringUtils.isNotBlank(string)) {
    +                String[] strings = string.split("\\s+");
    +                rets.addAll(Arrays.asList(strings));
    +            }
    +        } else if (value instanceof List) {
    +            @SuppressWarnings("unchecked")
    +            List<String> objects = (List<String>) value;
    +            for (String object : objects) {
    +                String str = substituteChildOptsInternal(object, memOnheap);
    +                if (StringUtils.isNotBlank(str)) {
    +                    rets.add(str);
    +                }
    +            }
    +        }
    +        return rets;
    +    }
    +
    +    /**
    +     * Launch the worker process (non-blocking)
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param processExitcallback
    +     *            a callback for when the process exits
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     */
    +    protected void launchWorkerProcess(List<String> command, Map<String, String> env, String logPrefix,
    +            ExitCodeCallback processExitCallback, File targetDir) throws IOException {
    +        if (_resourceIsolationManager != null) {
    +          command = _resourceIsolationManager.getLaunchCommand(_workerId, command);
    +        }
    +        SupervisorUtils.launchProcess(command, env, logPrefix, processExitCallback, targetDir);
    +    }
    +
    +    private String getWorkerLoggingConfigFile() {
    +        String log4jConfigurationDir = (String) (_conf.get(Config.STORM_LOG4J2_CONF_DIR));
    +
    +        if (StringUtils.isNotBlank(log4jConfigurationDir)) {
    +            if (!Utils.isAbsolutePath(log4jConfigurationDir)) {
    +                log4jConfigurationDir = _stormHome + Utils.FILE_PATH_SEPARATOR + log4jConfigurationDir;
    +            }
    +        } else {
    +            log4jConfigurationDir = _stormHome + Utils.FILE_PATH_SEPARATOR + "log4j2";
    +        }
    + 
    +        if (Utils.IS_ON_WINDOWS && !log4jConfigurationDir.startsWith("file:")) {
    +            log4jConfigurationDir = "file:///" + log4jConfigurationDir;
    +        }
    +        return log4jConfigurationDir + Utils.FILE_PATH_SEPARATOR + "worker.xml";
    +    }
    +    
    +    /**
    +     * Get parameters for the class path of the worker process.  Also used by the
    +     * log Writer
    +     * @param stormRoot the root dist dir for the topology
    +     * @return the classpath for the topology as command line arguments.
    +     * @throws IOException on any error.
    +     */
    +    private List<String> getClassPathParams(final String stormRoot) throws IOException {
    +        final String stormJar = ConfigUtils.supervisorStormJarPath(stormRoot);
    +        final StormTopology stormTopology = ConfigUtils.readSupervisorTopology(_conf, _topologyId, _ops);
    +        final List<String> dependencyLocations = new ArrayList<>();
    +        if (stormTopology.get_dependency_jars() != null) {
    +            for (String dependency : stormTopology.get_dependency_jars()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +
    +        if (stormTopology.get_dependency_artifacts() != null) {
    +            for (String dependency : stormTopology.get_dependency_artifacts()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +        final String workerClassPath = getWorkerClassPath(stormJar, dependencyLocations);
    +        
    +        List<String> classPathParams = new ArrayList<>();
    +        classPathParams.add("-cp");
    +        classPathParams.add(workerClassPath);
    +        return classPathParams;
    +    }
    +    
    +    /**
    +     * Get a set of java properties that are common to both the log writer and the worker processes.
    +     * These are mostly system properties that are used by logging.
    +     * @return a list of command line options
    +     */
    +    private List<String> getCommonParams() {
    +        final String workersArtifacts = ConfigUtils.workerArtifactsRoot(_conf);
    +        String stormLogDir = ConfigUtils.getLogDir();
    +        String log4jConfigurationFile = getWorkerLoggingConfigFile();
    +        
    +        List<String> commonParams = new ArrayList<>();
    +        commonParams.add("-Dlogging.sensitivity=" + OR((String) _topoConf.get(Config.TOPOLOGY_LOGGING_SENSITIVITY), "S3"));
    +        commonParams.add("-Dlogfile.name=worker.log");
    +        commonParams.add("-Dstorm.home=" + OR(_stormHome, ""));
    +        commonParams.add("-Dworkers.artifacts=" + workersArtifacts);
    +        commonParams.add("-Dstorm.id=" + _topologyId);
    +        commonParams.add("-Dworker.id=" + _workerId);
    +        commonParams.add("-Dworker.port=" + _port);
    +        commonParams.add("-Dstorm.log.dir=" + stormLogDir);
    +        commonParams.add("-Dlog4j.configurationFile=" + log4jConfigurationFile);
    +        commonParams.add("-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector");
    +        commonParams.add("-Dstorm.local.dir=" + _conf.get(Config.STORM_LOCAL_DIR));
    +        return commonParams;
    +    }
    +    
    +    private int getMemOnHeap(WorkerResources resources) {
    +        int memOnheap = 0;
    +        if (resources != null && resources.is_set_mem_on_heap() && 
    +                resources.get_mem_on_heap() > 0) {
    +            memOnheap = (int) Math.ceil(resources.get_mem_on_heap());
    +        } else {
    +            // set the default heap memory size for supervisor-test
    +            memOnheap = Utils.getInt(_topoConf.get(Config.WORKER_HEAP_MEMORY_MB), 768);
    +        }
    +        return memOnheap;
    +    }
    +    
    +    private List<String> getWorkerProfilerChildOpts(int memOnheap) {
    +        List<String> workerProfilerChildopts = new ArrayList<>();
    +        if (Utils.getBoolean(_conf.get(Config.WORKER_PROFILER_ENABLED), false)) {
    +            workerProfilerChildopts = substituteChildopts(_conf.get(Config.WORKER_PROFILER_CHILDOPTS), memOnheap);
    +        }
    +        return workerProfilerChildopts;
    +    }
    +    
    +    /**
    +     * a or b the first one that is not null
    +     * @param a something
    +     * @param b something else
    +     * @return a or b the first one that is not null
    +     */
    +    private <V> V OR(V a, V b) {
    +        return a == null ? b : a;
    +    }
    +    
    +    protected String javaCmd(String cmd) {
    +        String ret = null;
    +        String javaHome = System.getenv().get("JAVA_HOME");
    +        if (StringUtils.isNotBlank(javaHome)) {
    +            ret = javaHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR + cmd;
    +        } else {
    +            ret = cmd;
    +        }
    +        return ret;
    +    }
    +    
    +    /**
    +     * Create the command to launch the worker process
    +     * @param memOnheap the on heap memory for the worker
    +     * @param stormRoot the root dist dir for the topology
    +     * @param jlp java library path for the topology
    +     * @return the command to run
    +     * @throws IOException on any error.
    +     */
    +    private List<String> mkLaunchCommand(final int memOnheap, final String stormRoot,
    +            final String jlp) throws IOException {
    +        final String javaCmd = javaCmd("java");
    +        final String stormOptions = ConfigUtils.concatIfNotNull(System.getProperty("storm.options"));
    +        final String stormConfFile = ConfigUtils.concatIfNotNull(System.getProperty("storm.conf.file"));
    +        final String workerTmpDir = ConfigUtils.workerTmpRoot(_conf, _workerId);
    +        
    +        List<String> classPathParams = getClassPathParams(stormRoot);
    +        List<String> commonParams = getCommonParams();
    +        
    +        List<String> commandList = new ArrayList<>();
    +        //Log Writer Command...
    +        commandList.add(javaCmd);
    +        commandList.addAll(classPathParams);
    +        commandList.addAll(substituteChildopts(_topoConf.get(Config.TOPOLOGY_WORKER_LOGWRITER_CHILDOPTS)));
    +        commandList.addAll(commonParams);
    +        commandList.add("org.apache.storm.LogWriter"); //The LogWriter in turn launches the actual worker.
    +
    +        //Worker Command...
    +        commandList.add(javaCmd);
    +        commandList.add("-server");
    +        commandList.addAll(commonParams);
    +        commandList.addAll(substituteChildopts(_conf.get(Config.WORKER_CHILDOPTS), memOnheap));
    +        commandList.addAll(substituteChildopts(_topoConf.get(Config.TOPOLOGY_WORKER_CHILDOPTS), memOnheap));
    +        commandList.addAll(substituteChildopts(OR(
    +                _topoConf.get(Config.TOPOLOGY_WORKER_GC_CHILDOPTS),
    +                _conf.get(Config.WORKER_GC_CHILDOPTS)), memOnheap));
    +        commandList.addAll(getWorkerProfilerChildOpts(memOnheap));
    +        commandList.add("-Djava.library.path=" + jlp);
    +        commandList.add("-Dstorm.conf.file=" + stormConfFile);
    +        commandList.add("-Dstorm.options=" + stormOptions);
    +        commandList.add("-Djava.io.tmpdir=" + workerTmpDir);
    +        commandList.addAll(classPathParams);
    +        commandList.add("org.apache.storm.daemon.worker");
    +        commandList.add(_topologyId);
    +        commandList.add(_supervisorId);
    +        commandList.add(String.valueOf(_port));
    +        commandList.add(_workerId);
    +        
    +        return commandList;
    +    }
    +
    +    @Override
    +    public void launch() throws IOException {
    +        _type.assertFull();
    +        LOG.info("Launching worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                _supervisorId, _port, _workerId);
    +        String logPrefix = "Worker Process " + _workerId;
    +        ProcessExitCallback processExitCallback = new ProcessExitCallback(logPrefix);
    +        _exitedEarly = false;
    +        
    +        final WorkerResources resources = _assignment.get_resources();
    +        final int memOnheap = getMemOnHeap(resources);
    +        final String stormRoot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +        final String jlp = javaLibraryPath(stormRoot, _conf);
    +        
    +        List<String> commandList = mkLaunchCommand(memOnheap, stormRoot, jlp);
    +
    +        Map<String, String> topEnvironment = new HashMap<String, String>();
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> environment = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (environment != null) {
    +            topEnvironment.putAll(environment);
    +        }
    +        topEnvironment.put("LD_LIBRARY_PATH", jlp);
    --- End diff --
    
    Why is it necessary to set this in addition to java.library.path?


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76334407
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,202 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.nio.file.FileSystems;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    +    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
    +    
    +    /**
    +     * Factory to create a new AdvancedFSOps
    +     * @param conf the configuration of the process
    +     * @return the appropriate instance of the class for this config and environment.
    +     */
    +    public static AdvancedFSOps mk(Map<String, Object> conf) {
    +        if (Utils.isOnWindows()) {
    +            return new AdvancedWindowsFSOps(conf);
    +        }
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new AdvancedRunAsUserFSOps(conf);
    +        }
    +        return new AdvancedFSOps();
    +    }
    +    
    +    private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
    +        private final Map<String, Object> _conf;
    +        
    +        public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
    +            if (Utils.isOnWindows()) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +            _conf = conf;
    +        }
    +        
    +        @Override
    +        public void setupBlobPermissions(File path, String user) throws IOException {
    +            String logPrefix = "setup blob permissions for " + path;
    +            SupervisorUtils.processLauncherAndWait(_conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
    +        }
    +        
    +        public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +            String absolutePath = path.getAbsolutePath();
    +            LOG.debug("Deleting path {}", absolutePath);
    +            if (user == null) {
    +                user = Files.getOwner(path.toPath()).getName();
    +            }
    +            List<String> commands = new ArrayList<>();
    +            commands.add("rmr");
    +            commands.add(absolutePath);
    +            SupervisorUtils.processLauncherAndWait(_conf, user, commands, null, logPrefix);
    +            if (Utils.checkFileExists(absolutePath)) {
    +                throw new RuntimeException(path + " was not deleted.");
    +            }
    +        }
    +        
    +        public void setupStormCodeDir(Map<String, Object> topologyConf, String path) throws IOException {
    --- End diff --
    
    need Override tag


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77355233
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    --- End diff --
    
    minor: We define keys as Integer, but we are using Long for ports.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77367840
  
    --- 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 --
    
    Although we don't need to track start times for every state for functionality, it might be nice to keep track of start times if only to print them in log messages. This is a nice-to-have suggestion and not required now. But it might be easy to add.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77416395
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    --- End diff --
    
    We only do it in the constructor when the field names are the same.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76308272
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,494 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +    
    +    private class ProcessExitCallback implements Utils.ExitCodeCallable<Void> {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public Void call() throws Exception {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +            return null;
    +        }
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, LocalState localState, 
    +            ResourceIsolationInterface resourceIsolationManager, boolean recover) throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized(localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry: workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port +" "+ assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +        
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +    
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized(_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +        synchronized(_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            _localState.setApprovedWorkers(workersToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +    
    +    /**
    +     * Run the given command for profiling
    +     * @param command the command to run
    +     * @param env the environment to run the command
    +     * @param logPrefix the prefix to include in the logs
    +     * @param targetDir the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException on any error
    +     * @throws InterruptedException if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix, File targetDir) throws IOException, InterruptedException {
    +        Process p = Utils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +    
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String topologyId = _assignment.get_topology_id();
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, topologyId, _port);
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) topologyConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, topologyId, _port);
    +
    +        String workerPid = null;
    +        try (FileReader reader = new FileReader(str); BufferedReader br = new BufferedReader(reader)) {
    +            workerPid = br.readLine().trim();
    +        }
    +        
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        return runProfilingCommand(command, env, logPrefix, targetFile);
    +    }
    +    
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        if (action == ProfileAction.JMAP_DUMP) {
    +            return jmapDumpCmd(workerPid, targetDir);
    +        } else if (action == ProfileAction.JSTACK_DUMP) {
    +            return jstackDumpCmd(workerPid, targetDir);
    +        } else if (action == ProfileAction.JPROFILE_DUMP) {
    +            return jprofileDump(workerPid, targetDir);
    +        } else if (action == ProfileAction.JVM_RESTART) {
    +            return jprofileJvmRestart(workerPid);
    +        } else if (!stop && action == ProfileAction.JPROFILE_STOP) {
    +            return jprofileStart(workerPid);
    +        } else if (stop && action == ProfileAction.JPROFILE_STOP) {
    +            return jprofileStop(workerPid, targetDir);
    +        }
    +        return Lists.newArrayList();
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    --- End diff --
    
    Not sure what you mean by this.  Ideally if we had an enum that we could put a function like cmd in.  Then It would just be
    
    ```
    return action.cmd(workerPid, targetDir, stop);
    ```
    
    but ProfileAction is thrift generated code so we cannot play games like that here.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78238210
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,427 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.net.URLDecoder;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + */
    +public class AsyncLocalizer implements ILocalizer, Shutdownable {
    +    /**
    +     * A future that has already completed.
    +     */
    +    private static class AllDoneFuture implements Future<Void> {
    +
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isCancelled() {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isDone() {
    +            return true;
    +        }
    +
    +        @Override
    +        public Void get() {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void get(long timeout, TimeUnit unit) {
    +            return null;
    +        }
    +
    +    }
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizer.class);
    +
    +    private final Localizer _localizer;
    +    private final ExecutorService _execService;
    +    private final boolean _isLocalMode;
    +    private final Map<String, Object> _conf;
    +    private final Map<String, LocalDownloadedResource> _basicPending;
    +    private final Map<String, LocalDownloadedResource> _blobPending;
    +    private final AdvancedFSOps _fsOps;
    +
    +    private class DownloadBaseBlobsDistributed implements Callable<Void> {
    +        protected final String _topologyId;
    +        protected final File _stormRoot;
    +        
    +        public DownloadBaseBlobsDistributed(String topologyId) throws IOException {
    +            _topologyId = topologyId;
    +            _stormRoot = new File(ConfigUtils.supervisorStormDistRoot(_conf, _topologyId));
    +        }
    +        
    +        protected void downloadBaseBlobs(File tmproot) throws Exception {
    +            String stormJarKey = ConfigUtils.masterStormJarKey(_topologyId);
    +            String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +            String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +            String jarPath = ConfigUtils.supervisorStormJarPath(tmproot.getAbsolutePath());
    +            String codePath = ConfigUtils.supervisorStormCodePath(tmproot.getAbsolutePath());
    +            String confPath = ConfigUtils.supervisorStormConfPath(tmproot.getAbsolutePath());
    +            _fsOps.forceMkdir(tmproot);
    +            _fsOps.restrictDirectoryPermissions(tmproot);
    +            ClientBlobStore blobStore = Utils.getClientBlobStoreForSupervisor(_conf);
    +            try {
    +                Utils.downloadResourcesAsSupervisor(stormJarKey, jarPath, blobStore);
    +                Utils.downloadResourcesAsSupervisor(stormCodeKey, codePath, blobStore);
    +                Utils.downloadResourcesAsSupervisor(stormConfKey, confPath, blobStore);
    +            } finally {
    +                blobStore.shutdown();
    +            }
    +            Utils.extractDirFromJar(jarPath, ConfigUtils.RESOURCES_SUBDIR, tmproot);
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            if (_fsOps.fileExists(_stormRoot)) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    _fsOps.deleteIfExists(_stormRoot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            File tr = new File(tmproot);
    +            try {
    +                downloadBaseBlobs(tr);
    +                _fsOps.moveDirectoryPreferAtomic(tr, _stormRoot);
    +                _fsOps.setupStormCodeDir(ConfigUtils.readSupervisorStormConf(_conf, _topologyId), _stormRoot);
    +                deleteAll = false;
    +            } finally {
    +                if (deleteAll) {
    +                    LOG.warn("Failed to download basic resources for topology-id {}", _topologyId);
    +                    _fsOps.deleteIfExists(tr);
    +                    _fsOps.deleteIfExists(_stormRoot);
    +                }
    +            }
    +            return null;
    +        }
    +    }
    +    
    +    private class DownloadBaseBlobsLocal extends DownloadBaseBlobsDistributed {
    +
    +        public DownloadBaseBlobsLocal(String topologyId) throws IOException {
    +            super(topologyId);
    +        }
    +        
    +        @Override
    +        protected void downloadBaseBlobs(File tmproot) throws Exception {
    +            _fsOps.forceMkdir(tmproot);
    +            String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +            String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +            File codePath = new File(ConfigUtils.supervisorStormCodePath(tmproot.getAbsolutePath()));
    +            File confPath = new File(ConfigUtils.supervisorStormConfPath(tmproot.getAbsolutePath()));
    +            BlobStore blobStore = Utils.getNimbusBlobStore(_conf, null);
    +            try {
    +                try (OutputStream codeOutStream = _fsOps.getOutputStream(codePath)){
    +                    blobStore.readBlobTo(stormCodeKey, codeOutStream, null);
    +                }
    +                try (OutputStream confOutStream = _fsOps.getOutputStream(confPath)) {
    +                    blobStore.readBlobTo(stormConfKey, confOutStream, null);
    +                }
    +            } finally {
    +                blobStore.shutdown();
    +            }
    +
    +            ClassLoader classloader = Thread.currentThread().getContextClassLoader();
    +            String resourcesJar = AsyncLocalizer.resourcesJar();
    +            URL url = classloader.getResource(ConfigUtils.RESOURCES_SUBDIR);
    +
    +            String targetDir = tmproot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +
    +            if (resourcesJar != null) {
    +                LOG.info("Extracting resources from jar at {} to {}", resourcesJar, targetDir);
    +                Utils.extractDirFromJar(resourcesJar, ConfigUtils.RESOURCES_SUBDIR, _stormRoot);
    +            } else if (url != null) {
    +                LOG.info("Copying resources at {} to {} ", url.toString(), targetDir);
    +                if ("jar".equals(url.getProtocol())) {
    +                    JarURLConnection urlConnection = (JarURLConnection) url.openConnection();
    +                    Utils.extractDirFromJar(urlConnection.getJarFileURL().getFile(), ConfigUtils.RESOURCES_SUBDIR, _stormRoot);
    +                } else {
    +                    _fsOps.copyDirectory(new File(url.getFile()), new File(targetDir));
    +                }
    +            }
    +        }
    +    }
    +    
    +    private class DownloadBlobs implements Callable<Void> {
    +        private final String _topologyId;
    +
    +        public DownloadBlobs(String topologyId) {
    +            _topologyId = topologyId;
    +        }
    +
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            Map<String, Object> stormConf = ConfigUtils.readSupervisorStormConf(_conf, _topologyId);
    +                
    +            @SuppressWarnings("unchecked")
    +            Map<String, Map<String, Object>> blobstoreMap = (Map<String, Map<String, Object>>) stormConf.get(Config.TOPOLOGY_BLOBSTORE_MAP);
    +            String user = (String) stormConf.get(Config.TOPOLOGY_SUBMITTER_USER);
    +            String topoName = (String) stormConf.get(Config.TOPOLOGY_NAME);
    +
    +            List<LocalResource> localResourceList = new ArrayList<>();
    +            if (blobstoreMap != null) {
    +                List<LocalResource> tmp = SupervisorUtils.blobstoreMapToLocalresources(blobstoreMap);
    +                if (tmp != null) {
    +                    localResourceList.addAll(tmp);
    +                }
    +            }
    +
    +            StormTopology stormCode = ConfigUtils.readSupervisorTopology(_conf, _topologyId, _fsOps);
    +            List<String> dependencies = new ArrayList<>();
    +            if (stormCode.is_set_dependency_jars()) {
    +                dependencies.addAll(stormCode.get_dependency_jars());
    +            }
    +            if (stormCode.is_set_dependency_artifacts()) {
    +                dependencies.addAll(stormCode.get_dependency_artifacts());
    +            }
    +            for (String dependency : dependencies) {
    +                localResourceList.add(new LocalResource(dependency, false));
    +            }
    +            
    +            if (!localResourceList.isEmpty()) {
    +                File userDir = _localizer.getLocalUserFileCacheDir(user);
    +                if (!_fsOps.fileExists(userDir)) {
    +                    _fsOps.forceMkdir(userDir);
    +                }
    +                List<LocalizedResource> localizedResources = _localizer.getBlobs(localResourceList, user, topoName, userDir);
    +                _fsOps.setupBlobPermissions(userDir, user);
    +                for (LocalizedResource localizedResource : localizedResources) {
    +                    String keyName = localizedResource.getKey();
    +                    //The sym link we are pointing to
    +                    File rsrcFilePath = new File(localizedResource.getCurrentSymlinkPath());
    +
    +                    String symlinkName = null;
    +                    Map<String, Object> blobInfo = blobstoreMap.get(keyName);
    +                    if (blobInfo != null && blobInfo.containsKey("localname")) {
    +                        symlinkName = (String) blobInfo.get("localname");
    +                    } else {
    +                        symlinkName = keyName;
    +                    }
    +                    _fsOps.createSymlink(new File(stormroot, symlinkName), rsrcFilePath);
    +                }
    +            }
    +
    +            return null;
    +        }
    +    }
    +    
    +    //Visible for testing
    +    AsyncLocalizer(Map<String, Object> conf, Localizer localizer, AdvancedFSOps ops) {
    +        _conf = conf;
    +        _isLocalMode = ConfigUtils.isLocalMode(conf);
    +        _localizer = localizer;
    +        _execService = Executors.newFixedThreadPool(1,  
    +                new ThreadFactoryBuilder()
    +                .setNameFormat("Async Localizer")
    +                .build());
    +        _basicPending = new HashMap<>();
    +        _blobPending = new HashMap<>();
    +        _fsOps = ops;
    +    }
    +    
    +    public AsyncLocalizer(Map<String, Object> conf, Localizer localizer) {
    +        this(conf, localizer, AdvancedFSOps.make(conf));
    +    }
    +
    +    @Override
    +    public synchronized Future<Void> requestDownloadBaseTopologyBlobs(final LocalAssignment assignment, final int port) throws IOException {
    +        final String topologyId = assignment.get_topology_id();
    +        LocalDownloadedResource localResource = _basicPending.get(topologyId);
    +        if (localResource == null) {
    +            Callable<Void> c;
    +            if (_isLocalMode) {
    +                c = new DownloadBaseBlobsLocal(topologyId);
    +            } else {
    +                c = new DownloadBaseBlobsDistributed(topologyId);
    +            }
    +            localResource = new LocalDownloadedResource(_execService.submit(c));
    +            _basicPending.put(topologyId, localResource);
    +        }
    +        Future<Void> ret = localResource.reserve(port, assignment);
    +        LOG.debug("Reserved basic {} {}", topologyId, localResource);
    +        return ret;
    +    }
    +
    +    private static String resourcesJar() throws IOException {
    +        String path = Utils.currentClasspath();
    +        if (path == null) {
    +            return null;
    +        }
    +        String[] paths = path.split(File.pathSeparator);
    +        List<String> jarPaths = new ArrayList<String>();
    +        for (String s : paths) {
    +            if (s.endsWith(".jar")) {
    +                jarPaths.add(s);
    +            }
    +        }
    +
    +        List<String> rtn = new ArrayList<String>();
    +        int size = jarPaths.size();
    +        for (int i = 0; i < size; i++) {
    --- End diff --
    
    how about: for (String path : jarPaths)


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @harshach There's already an another pull request for fixing race condition on old supervisor. Given that we can't set a due date for 2.0 (and maintenance date for 1.x) it would be better to have this as kind of bugfix.
    
    @revans2 The code block in my comment fixes the issue. You seem to fix it, but it's gone now.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77415710
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    +                            if (actions == null) {
    +                                actions = new HashSet<>();
    +                                filtered.put(port.intValue(), actions);
    +                            }
    +                            actions.add(new TopoProfileAction(topoId, req));
    +                        }
    +                    }
    +                }
    +            }
    +            
    +            for (Integer port: allPorts) {
    +                Slot slot = slots.get(port);
    +                if (slot == null) {
    +                    slot = mkSlot(port);
    +                    slots.put(port, slot);
    +                }
    +                slot.setNewAssignment(allAssignments.get(port));
    +                slot.addProfilerActions(filtered.get(port));
    +            }
    +            
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +    }
    +    
    +    protected Map<String, VersionedData<Assignment>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +            Map<String, VersionedData<Assignment>> localAssignmentVersion, Runnable callback) throws Exception {
    +        Map<String, VersionedData<Assignment>> updateAssignmentVersion = new HashMap<>();
    +        for (String topoId : topoIds) {
    +            Integer recordedVersion = -1;
    +            Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +            VersionedData<Assignment> locAssignment = localAssignmentVersion.get(topoId);
    +            if (locAssignment != null) {
    +                recordedVersion = locAssignment.getVersion();
    +            }
    +            if (version == null) {
    +                // ignore
    +            } else if (version == recordedVersion) {
    +                updateAssignmentVersion.put(topoId, locAssignment);
    +            } else {
    +                VersionedData<Assignment> assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                updateAssignmentVersion.put(topoId, assignmentVersion);
    +            }
    +        }
    +        return updateAssignmentVersion;
    +    }
    +    
    +    protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +        Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +        for (String stormId : stormIds) {
    +            List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +            ret.put(stormId, profileRequests);
    +        }
    +        return ret;
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readAssignments(Map<String, VersionedData<Assignment>> assignmentsSnapshot,
    +            String assignmentId, AtomicInteger retries) {
    +        try {
    +            Map<Integer, LocalAssignment> portLA = new HashMap<Integer, LocalAssignment>();
    +            for (Map.Entry<String, VersionedData<Assignment>> assignEntry : assignmentsSnapshot.entrySet()) {
    +                String topoId = assignEntry.getKey();
    +                Assignment assignment = assignEntry.getValue().getData();
    +
    +                Map<Integer, LocalAssignment> portTasks = readMyExecutors(topoId, assignmentId, assignment);
    +
    +                for (Map.Entry<Integer, LocalAssignment> entry : portTasks.entrySet()) {
    +
    +                    Integer port = entry.getKey();
    +
    +                    LocalAssignment la = entry.getValue();
    +
    +                    if (!portLA.containsKey(port)) {
    +                        portLA.put(port, la);
    +                    } else {
    +                        throw new RuntimeException("Should not have multiple topologies assigned to one port");
    +                    }
    +                }
    +            }
    +            retries.set(0);
    +            return portLA;
    +        } catch (RuntimeException e) {
    +            if (retries.get() > 2) {
    +                throw e;
    +            } else {
    +                retries.addAndGet(1);
    +            }
    +            LOG.warn("{} : retrying {} of 3", e.getMessage(), retries.get());
    +            return null;
    +        }
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readMyExecutors(String stormId, String assignmentId, Assignment assignment) {
    +        Map<Integer, LocalAssignment> portTasks = new HashMap<>();
    +        Map<Long, WorkerResources> slotsResources = new HashMap<>();
    +        Map<NodeInfo, WorkerResources> nodeInfoWorkerResourcesMap = assignment.get_worker_resources();
    +        if (nodeInfoWorkerResourcesMap != null) {
    +            for (Map.Entry<NodeInfo, WorkerResources> entry : nodeInfoWorkerResourcesMap.entrySet()) {
    +                if (entry.getKey().get_node().equals(assignmentId)) {
    +                    Set<Long> ports = entry.getKey().get_port();
    +                    for (Long port : ports) {
    +                        slotsResources.put(port, entry.getValue());
    +                    }
    +                }
    +            }
    +        }
    +        Map<List<Long>, NodeInfo> executorNodePort = assignment.get_executor_node_port();
    +        if (executorNodePort != null) {
    +            for (Map.Entry<List<Long>, NodeInfo> entry : executorNodePort.entrySet()) {
    +                if (entry.getValue().get_node().equals(assignmentId)) {
    +                    for (Long port : entry.getValue().get_port()) {
    +                        LocalAssignment localAssignment = portTasks.get(port.intValue());
    +                        if (localAssignment == null) {
    +                            List<ExecutorInfo> executors = new ArrayList<ExecutorInfo>();
    +                            localAssignment = new LocalAssignment(stormId, executors);
    +                            if (slotsResources.containsKey(port)) {
    +                                localAssignment.set_resources(slotsResources.get(port));
    +                            }
    +                            portTasks.put(port.intValue(), localAssignment);
    +                        }
    +                        List<ExecutorInfo> executorInfoList = localAssignment.get_executors();
    +                        executorInfoList.add(new ExecutorInfo(entry.getKey().get(0).intValue(), entry.getKey().get(entry.getKey().size() - 1).intValue()));
    +                    }
    +                }
    +            }
    +        }
    +        return portTasks;
    +    }
    +
    +    public synchronized void shutdownAllWorkers() {
    +        for (Slot slot: slots.values()) {
    +            try {
    +                slot.setNewAssignment(null);
    +            } catch (Exception e) {
    +                LOG.error("Error trying to shutdown workers in {}", slot, e);
    +            }
    +        }
    +
    +        for (Slot slot: slots.values()) {
    +            try {
    +                int count = 0;
    +                while (slot.getMachineState() != MachineState.EMPTY) {
    +                    if (count > 10) {
    +                        LOG.warn("DONE waiting for {} to finish {}", slot, slot.getMachineState());
    +                        break;
    +                    }
    +                    if (Time.isSimulating()) {
    +                        Time.advanceTime(1000);
    +                        Thread.sleep(100);
    --- End diff --
    
    We are simulating, so in general we don't actually wait as long in simulated time, to speed up the tests.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @srdo Ya that is what I did.  Thanks for the advice.  Not sure what happened somehow when I upmerged all but 2 of my commits disappeared.  


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78965321
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/RunAsUserContainer.java ---
    @@ -0,0 +1,94 @@
    +/**
    + * 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.File;
    +import java.io.IOException;
    +import java.util.Arrays;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class RunAsUserContainer extends BasicContainer {
    +    private static final Logger LOG = LoggerFactory.getLogger(RunAsUserContainer.class);
    +
    +    public RunAsUserContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int port,
    +            LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager, LocalState localState,
    +            String workerId, Map<String, Object> topoConf, AdvancedFSOps ops, String profileCmd) throws IOException {
    +        super(type, conf, supervisorId, port, assignment, resourceIsolationManager, localState, workerId, topoConf, ops,
    +                profileCmd);
    +        if (Utils.isOnWindows()) {
    +            throw new UnsupportedOperationException("ERROR: Windows doesn't support running workers as different users yet");
    +        }
    +    }
    +
    +    private void signal(long pid, int signal) throws IOException {
    +        List<String> commands = Arrays.asList("signal", String.valueOf(pid), String.valueOf(signal));
    +        String user = getWorkerUser();
    +        String logPrefix = "kill -"+signal+" " + pid;
    +        SupervisorUtils.processLauncherAndWait(_conf, user, commands, null, logPrefix);
    +    }
    +    
    +    @Override
    +    protected void kill(long pid) throws IOException {
    +        signal(pid, 15);
    +    }
    +    
    +    @Override
    +    protected void forceKill(long pid) throws IOException {
    +        signal(pid, 9);
    +    }
    +    
    +    @Override
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix, File targetDir) throws IOException, InterruptedException {
    +        String user = this.getWorkerUser();
    +        String td = targetDir.getAbsolutePath();
    +        LOG.info("Running as user:{} command:{}", user, command);
    --- End diff --
    
    Nitpick: Might want to add spaces between the colon and braces


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    I merged this in my local, and do some tests, and see:
    
    1. integration test fails from my dev. machine
    
    - I'm using OSX 10.11, Java 1.8.0_66
    - I got wrong IP address from integration test. I'm using IP sharing router but assigned IP is even not same as external IP.
    -- This is similar to what I consistently observed from my shared office. 
    -- In my office even build for current master is failing. (I'm testing this pull request in home now.)
    -- Strange thing is that normal unit tests are not failing for this build. In my office it's also failing.
    
    2. There's a bug for creating symbolic link for dependencies, and also createSymbolicLink.  I'll leave a comment to diff.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77526151
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    --- End diff --
    
    Happy to be wrong here, it is a lot smaller.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79034214
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected final String _stormHome = System.getProperty("storm.home");
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +    
    +    /**
    +     * Create a new BasicContainer
    +     * @param type the type of container being made.
    +     * @param conf the supervisor config
    +     * @param supervisorId the ID of the supervisor this is a part of.
    +     * @param port the port the container is on.  Should be <= 0 if only a partial recovery
    +     * @param assignment the assignment for this container. Should be null if only a partial recovery.
    +     * @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used.
    +     * @param localState the local state of the supervisor.  May be null if partial recovery
    +     * @param workerId the id of the worker to use.  Must not be null if doing a partial recovery.
    +     * @param ops file system operations (mostly for testing) if null a new one is made
    +     * @param topoConf the config of the topology (mostly for testing) if null 
    +     * and not a partial recovery the real conf is read.
    +     * @param profileCmd the command to use when profiling (used for testing)
    +     */
    +    public BasicContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int port,
    +            LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
    +            LocalState localState, String workerId, Map<String, Object> topoConf, 
    +            AdvancedFSOps ops, String profileCmd) throws IOException {
    +        super(type, conf, supervisorId, port, assignment, resourceIsolationManager, workerId, topoConf, ops);
    +        assert(localState != null);
    +        _localState = localState;
    +
    +        if (type.isRecovery() && !type.isOnlyKillable()) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                LOG.info("Recovered Worker {}", wid);
    +                _workerId = wid;
    +            }
    +        } else if (_workerId == null){
    +            createNewWorkerId();
    +        }
    +
    +        if (profileCmd == null) {
    +            profileCmd = _stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                    + conf.get(Config.WORKER_PROFILER_COMMAND);
    +        }
    +        _profileCmd = profileCmd;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        _type.assertFull();
    +        assert(_workerId == null);
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            removeWorkersOn(workerToPort, _port);
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +            LOG.info("Created Worker ID {}", _workerId);
    +        }
    +    }
    +
    +    private static void removeWorkersOn(Map<String, Integer> workerToPort, int _port) {
    +        for (Iterator<Entry<String, Integer>> i = workerToPort.entrySet().iterator(); i.hasNext();) {
    +            Entry<String, Integer> found = i.next();
    +            if (_port == found.getValue().intValue()) {
    +                LOG.warn("Deleting worker {} from state", found.getKey());
    +                i.remove();
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUpForRestart() throws IOException {
    +        String origWorkerId = _workerId;
    +        super.cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(origWorkerId);
    +            removeWorkersOn(workersToPort, _port);
    +            _localState.setApprovedWorkers(workersToPort);
    +            LOG.info("Removed Worker ID {}", origWorkerId);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        _type.assertFull();
    +        //We are launching it now...
    +        _type = ContainerType.LAUNCH;
    +        createNewWorkerId();
    +        setup();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        _type.assertFull();
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        _type.assertFull();
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = _ops.slurpString(new File(str)).trim();
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        if (command.size() > 0) {
    +            return runProfilingCommand(command, env, logPrefix, targetFile);
    +        }
    +        LOG.warn("PROFILING REQUEST NOT SUPPORTED {} IGNORED...", request);
    +        return true;
    +    }
    +
    +    /**
    +     * Get the command to run when doing profiling
    +     * @param action the profiling action to perform
    +     * @param stop if this is meant to stop the profiling or start it
    +     * @param workerPid the PID of the process to profile
    +     * @param targetDir the current working directory of the worker process
    +     * @return the command to run for profiling.
    +     */
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        switch(action) {
    +            case JMAP_DUMP:
    +                return jmapDumpCmd(workerPid, targetDir);
    +            case JSTACK_DUMP:
    +                return jstackDumpCmd(workerPid, targetDir);
    +            case JPROFILE_DUMP:
    +                return jprofileDump(workerPid, targetDir);
    +            case JVM_RESTART:
    +                return jprofileJvmRestart(workerPid);
    +            case JPROFILE_STOP:
    +                if (stop) {
    +                    return jprofileStop(workerPid, targetDir);
    +                }
    +                return jprofileStart(workerPid);
    +            default:
    +                return Lists.newArrayList();
    +        }
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +
    +    /**
    +     * Compute the java.library.path that should be used for the worker.
    +     * This helps it to load JNI libraries that are packaged in the uber jar.
    +     * @param stormRoot the root directory of the worker process
    +     * @param conf the config for the supervisor.
    +     * @return the java.library.path/LD_LIBRARY_PATH to use so native libraries load correctly.
    +     */
    +    protected String javaLibraryPath(String stormRoot, Map<String, Object> conf) {
    +        String resourceRoot = stormRoot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +        String os = System.getProperty("os.name").replaceAll("\\s+", "_");
    +        String arch = System.getProperty("os.arch");
    +        String archResourceRoot = resourceRoot + Utils.FILE_PATH_SEPARATOR + os + "-" + arch;
    +        String ret = CPJ.join(archResourceRoot, resourceRoot,
    +                conf.get(Config.JAVA_LIBRARY_PATH));
    +        return ret;
    +    }
    +
    +    /**
    +     * Returns a collection of jar file names found under the given directory.
    +     * @param dir the directory to search
    +     * @return the jar file names
    +     */
    +    protected List<String> getFullJars(File dir) {
    +        File[] files = dir.listFiles(jarFilter);
    +
    +        if (files == null) {
    +            return Collections.emptyList();
    +        }
    +
    +        return Arrays.stream(files).map(f -> f.getAbsolutePath())
    +                .collect(Collectors.toList());
    +    }
    +    
    +    protected List<String> frameworkClasspath() {
    +        File stormLibDir = new File(_stormHome, "lib");
    +        String stormConfDir =
    +                System.getenv("STORM_CONF_DIR") != null ?
    +                System.getenv("STORM_CONF_DIR") :
    +                new File(_stormHome, "conf").getAbsolutePath();
    +        File stormExtlibDir = new File(_stormHome, "extlib");
    +        String extcp = System.getenv("STORM_EXT_CLASSPATH");
    +        List<String> pathElements = new LinkedList<>();
    +        pathElements.addAll(getFullJars(stormLibDir));
    +        pathElements.addAll(getFullJars(stormExtlibDir));
    +        pathElements.add(extcp);
    +        pathElements.add(stormConfDir);
    +
    +        return pathElements;
    +    }
    +    
    +    @SuppressWarnings("unchecked")
    +    private List<String> asStringList(Object o) {
    +        if (o instanceof String) {
    +            return Arrays.asList((String)o);
    +        } else if (o instanceof List) {
    +            return (List<String>)o;
    +        }
    +        return Collections.EMPTY_LIST;
    +    }
    +    
    +    /**
    +     * Compute the classpath for the worker process
    +     * @param stormJar the topology jar
    +     * @param dependencyLocations any dependencies from the topology
    +     * @return the full classpath
    +     */
    +    protected String getWorkerClassPath(String stormJar, List<String> dependencyLocations) {
    +        List<String> workercp = new ArrayList<>();
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH_BEGINNING)));
    +        workercp.addAll(frameworkClasspath());
    +        workercp.add(stormJar);
    +        workercp.addAll(dependencyLocations);
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH)));
    +        return CPJ.join(workercp);
    +    }
    +
    +    private String substituteChildOptsInternal(String string, int memOnheap) {
    +        if (StringUtils.isNotBlank(string)) {
    +            String p = String.valueOf(_port);
    +            string = string.replace("%ID%", p);
    +            string = string.replace("%WORKER-ID%", _workerId);
    +            string = string.replace("%TOPOLOGY-ID%", _topologyId);
    +            string = string.replace("%WORKER-PORT%", p);
    +            if (memOnheap > 0) {
    +                string = string.replace("%HEAP-MEM%", String.valueOf(memOnheap));
    +            }
    +        }
    +        return string;
    +    }
    +    
    +    protected List<String> substituteChildopts(Object value) {
    +        return substituteChildopts(value, -1);
    +    }
    +
    +    protected List<String> substituteChildopts(Object value, int memOnheap) {
    +        List<String> rets = new ArrayList<>();
    +        if (value instanceof String) {
    +            String string = substituteChildOptsInternal((String) value, memOnheap);
    +            if (StringUtils.isNotBlank(string)) {
    +                String[] strings = string.split("\\s+");
    +                rets.addAll(Arrays.asList(strings));
    +            }
    +        } else if (value instanceof List) {
    +            @SuppressWarnings("unchecked")
    +            List<String> objects = (List<String>) value;
    +            for (String object : objects) {
    +                String str = substituteChildOptsInternal(object, memOnheap);
    +                if (StringUtils.isNotBlank(str)) {
    +                    rets.add(str);
    +                }
    +            }
    +        }
    +        return rets;
    +    }
    +
    +    /**
    +     * Launch the worker process (non-blocking)
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param processExitcallback
    +     *            a callback for when the process exits
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     */
    +    protected void launchWorkerProcess(List<String> command, Map<String, String> env, String logPrefix,
    +            ExitCodeCallback processExitCallback, File targetDir) throws IOException {
    +        if (_resourceIsolationManager != null) {
    +          command = _resourceIsolationManager.getLaunchCommand(_workerId, command);
    +        }
    +        SupervisorUtils.launchProcess(command, env, logPrefix, processExitCallback, targetDir);
    +    }
    +
    +    private String getWorkerLoggingConfigFile() {
    +        String log4jConfigurationDir = (String) (_conf.get(Config.STORM_LOG4J2_CONF_DIR));
    +
    +        if (StringUtils.isNotBlank(log4jConfigurationDir)) {
    +            if (!Utils.isAbsolutePath(log4jConfigurationDir)) {
    +                log4jConfigurationDir = _stormHome + Utils.FILE_PATH_SEPARATOR + log4jConfigurationDir;
    +            }
    +        } else {
    +            log4jConfigurationDir = _stormHome + Utils.FILE_PATH_SEPARATOR + "log4j2";
    +        }
    + 
    +        if (Utils.IS_ON_WINDOWS && !log4jConfigurationDir.startsWith("file:")) {
    +            log4jConfigurationDir = "file:///" + log4jConfigurationDir;
    +        }
    +        return log4jConfigurationDir + Utils.FILE_PATH_SEPARATOR + "worker.xml";
    +    }
    +    
    +    /**
    +     * Get parameters for the class path of the worker process.  Also used by the
    +     * log Writer
    +     * @param stormRoot the root dist dir for the topology
    +     * @return the classpath for the topology as command line arguments.
    +     * @throws IOException on any error.
    +     */
    +    private List<String> getClassPathParams(final String stormRoot) throws IOException {
    +        final String stormJar = ConfigUtils.supervisorStormJarPath(stormRoot);
    +        final StormTopology stormTopology = ConfigUtils.readSupervisorTopology(_conf, _topologyId, _ops);
    +        final List<String> dependencyLocations = new ArrayList<>();
    +        if (stormTopology.get_dependency_jars() != null) {
    +            for (String dependency : stormTopology.get_dependency_jars()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +
    +        if (stormTopology.get_dependency_artifacts() != null) {
    +            for (String dependency : stormTopology.get_dependency_artifacts()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +        final String workerClassPath = getWorkerClassPath(stormJar, dependencyLocations);
    +        
    +        List<String> classPathParams = new ArrayList<>();
    +        classPathParams.add("-cp");
    +        classPathParams.add(workerClassPath);
    +        return classPathParams;
    +    }
    +    
    +    /**
    +     * Get a set of java properties that are common to both the log writer and the worker processes.
    +     * These are mostly system properties that are used by logging.
    +     * @return a list of command line options
    +     */
    +    private List<String> getCommonParams() {
    +        final String workersArtifacts = ConfigUtils.workerArtifactsRoot(_conf);
    +        String stormLogDir = ConfigUtils.getLogDir();
    +        String log4jConfigurationFile = getWorkerLoggingConfigFile();
    +        
    +        List<String> commonParams = new ArrayList<>();
    +        commonParams.add("-Dlogging.sensitivity=" + OR((String) _topoConf.get(Config.TOPOLOGY_LOGGING_SENSITIVITY), "S3"));
    +        commonParams.add("-Dlogfile.name=worker.log");
    +        commonParams.add("-Dstorm.home=" + OR(_stormHome, ""));
    +        commonParams.add("-Dworkers.artifacts=" + workersArtifacts);
    +        commonParams.add("-Dstorm.id=" + _topologyId);
    +        commonParams.add("-Dworker.id=" + _workerId);
    +        commonParams.add("-Dworker.port=" + _port);
    +        commonParams.add("-Dstorm.log.dir=" + stormLogDir);
    +        commonParams.add("-Dlog4j.configurationFile=" + log4jConfigurationFile);
    +        commonParams.add("-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector");
    +        commonParams.add("-Dstorm.local.dir=" + _conf.get(Config.STORM_LOCAL_DIR));
    +        return commonParams;
    +    }
    +    
    +    private int getMemOnHeap(WorkerResources resources) {
    +        int memOnheap = 0;
    +        if (resources != null && resources.is_set_mem_on_heap() && 
    +                resources.get_mem_on_heap() > 0) {
    +            memOnheap = (int) Math.ceil(resources.get_mem_on_heap());
    +        } else {
    +            // set the default heap memory size for supervisor-test
    +            memOnheap = Utils.getInt(_topoConf.get(Config.WORKER_HEAP_MEMORY_MB), 768);
    +        }
    +        return memOnheap;
    +    }
    +    
    +    private List<String> getWorkerProfilerChildOpts(int memOnheap) {
    +        List<String> workerProfilerChildopts = new ArrayList<>();
    +        if (Utils.getBoolean(_conf.get(Config.WORKER_PROFILER_ENABLED), false)) {
    +            workerProfilerChildopts = substituteChildopts(_conf.get(Config.WORKER_PROFILER_CHILDOPTS), memOnheap);
    +        }
    +        return workerProfilerChildopts;
    +    }
    +    
    +    /**
    +     * a or b the first one that is not null
    +     * @param a something
    +     * @param b something else
    +     * @return a or b the first one that is not null
    +     */
    +    private <V> V OR(V a, V b) {
    +        return a == null ? b : a;
    +    }
    +    
    +    protected String javaCmd(String cmd) {
    +        String ret = null;
    +        String javaHome = System.getenv().get("JAVA_HOME");
    +        if (StringUtils.isNotBlank(javaHome)) {
    +            ret = javaHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR + cmd;
    +        } else {
    +            ret = cmd;
    +        }
    +        return ret;
    +    }
    +    
    +    /**
    +     * Create the command to launch the worker process
    +     * @param memOnheap the on heap memory for the worker
    +     * @param stormRoot the root dist dir for the topology
    +     * @param jlp java library path for the topology
    +     * @return the command to run
    +     * @throws IOException on any error.
    +     */
    +    private List<String> mkLaunchCommand(final int memOnheap, final String stormRoot,
    +            final String jlp) throws IOException {
    +        final String javaCmd = javaCmd("java");
    +        final String stormOptions = ConfigUtils.concatIfNotNull(System.getProperty("storm.options"));
    +        final String stormConfFile = ConfigUtils.concatIfNotNull(System.getProperty("storm.conf.file"));
    +        final String workerTmpDir = ConfigUtils.workerTmpRoot(_conf, _workerId);
    +        
    +        List<String> classPathParams = getClassPathParams(stormRoot);
    +        List<String> commonParams = getCommonParams();
    +        
    +        List<String> commandList = new ArrayList<>();
    +        //Log Writer Command...
    +        commandList.add(javaCmd);
    +        commandList.addAll(classPathParams);
    +        commandList.addAll(substituteChildopts(_topoConf.get(Config.TOPOLOGY_WORKER_LOGWRITER_CHILDOPTS)));
    +        commandList.addAll(commonParams);
    +        commandList.add("org.apache.storm.LogWriter"); //The LogWriter in turn launches the actual worker.
    +
    +        //Worker Command...
    +        commandList.add(javaCmd);
    +        commandList.add("-server");
    +        commandList.addAll(commonParams);
    +        commandList.addAll(substituteChildopts(_conf.get(Config.WORKER_CHILDOPTS), memOnheap));
    +        commandList.addAll(substituteChildopts(_topoConf.get(Config.TOPOLOGY_WORKER_CHILDOPTS), memOnheap));
    +        commandList.addAll(substituteChildopts(OR(
    +                _topoConf.get(Config.TOPOLOGY_WORKER_GC_CHILDOPTS),
    +                _conf.get(Config.WORKER_GC_CHILDOPTS)), memOnheap));
    +        commandList.addAll(getWorkerProfilerChildOpts(memOnheap));
    +        commandList.add("-Djava.library.path=" + jlp);
    +        commandList.add("-Dstorm.conf.file=" + stormConfFile);
    +        commandList.add("-Dstorm.options=" + stormOptions);
    +        commandList.add("-Djava.io.tmpdir=" + workerTmpDir);
    +        commandList.addAll(classPathParams);
    +        commandList.add("org.apache.storm.daemon.worker");
    +        commandList.add(_topologyId);
    +        commandList.add(_supervisorId);
    +        commandList.add(String.valueOf(_port));
    +        commandList.add(_workerId);
    +        
    +        return commandList;
    +    }
    +
    +    @Override
    +    public void launch() throws IOException {
    +        _type.assertFull();
    +        LOG.info("Launching worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                _supervisorId, _port, _workerId);
    +        String logPrefix = "Worker Process " + _workerId;
    +        ProcessExitCallback processExitCallback = new ProcessExitCallback(logPrefix);
    +        _exitedEarly = false;
    +        
    +        final WorkerResources resources = _assignment.get_resources();
    +        final int memOnheap = getMemOnHeap(resources);
    +        final String stormRoot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +        final String jlp = javaLibraryPath(stormRoot, _conf);
    +        
    +        List<String> commandList = mkLaunchCommand(memOnheap, stormRoot, jlp);
    +
    +        Map<String, String> topEnvironment = new HashMap<String, String>();
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> environment = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (environment != null) {
    +            topEnvironment.putAll(environment);
    +        }
    +        topEnvironment.put("LD_LIBRARY_PATH", jlp);
    --- End diff --
    
    Makes sense.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77355840
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    --- End diff --
    
    Is it worth adding a debug-level log here?


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76464179
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,202 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.nio.file.FileSystems;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    +    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
    +    
    +    /**
    +     * Factory to create a new AdvancedFSOps
    +     * @param conf the configuration of the process
    +     * @return the appropriate instance of the class for this config and environment.
    +     */
    +    public static AdvancedFSOps mk(Map<String, Object> conf) {
    +        if (Utils.isOnWindows()) {
    +            return new AdvancedWindowsFSOps(conf);
    +        }
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new AdvancedRunAsUserFSOps(conf);
    +        }
    +        return new AdvancedFSOps();
    +    }
    +    
    +    private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
    +        private final Map<String, Object> _conf;
    +        
    +        public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
    +            if (Utils.isOnWindows()) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +            _conf = conf;
    +        }
    +        
    +        @Override
    +        public void setupBlobPermissions(File path, String user) throws IOException {
    +            String logPrefix = "setup blob permissions for " + path;
    +            SupervisorUtils.processLauncherAndWait(_conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
    +        }
    +        
    +        public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +            String absolutePath = path.getAbsolutePath();
    +            LOG.debug("Deleting path {}", absolutePath);
    +            if (user == null) {
    +                user = Files.getOwner(path.toPath()).getName();
    +            }
    +            List<String> commands = new ArrayList<>();
    +            commands.add("rmr");
    +            commands.add(absolutePath);
    +            SupervisorUtils.processLauncherAndWait(_conf, user, commands, null, logPrefix);
    +            if (Utils.checkFileExists(absolutePath)) {
    +                throw new RuntimeException(path + " was not deleted.");
    +            }
    +        }
    +        
    +        public void setupStormCodeDir(Map<String, Object> topologyConf, String path) throws IOException {
    +            SupervisorUtils.setupStormCodeDir(_conf, topologyConf, path);
    +        }
    +    }
    +    
    +    /**
    +     * Operations that need to override the default ones when running on Windows
    +     *
    +     */
    +    private static class AdvancedWindowsFSOps extends AdvancedFSOps {
    +
    +        public AdvancedWindowsFSOps(Map<String, Object> conf) {
    +            if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +        }
    +        
    +        @Override
    +        public void restrictDirectoryPermissions(String dir) throws IOException {
    +            //NOOP, if windows gets support for run as user we will need to find a way to suppor this
    +        }
    +        
    +        @Override
    +        public void moveDriectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // This is not atomic but it does work
    +            FileUtils.moveDirectory(fromDir, toDir);
    +        }
    +        
    +        @Override
    +        public boolean supportsAtomicDirectoryMove() {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // FileUtils.moveDirectory is not atomic
    +            return false;
    +        }
    +    }
    +    
    +    
    +    protected AdvancedFSOps() {
    +        //NOOP, but restricted permissions
    +    }
    +
    +    /**
    +     * Set directory permissions to (OWNER)RWX (GROUP)R-X (OTHER)---
    +     * On some systems that do not support this, it may become a noop
    +     * @param dir the directory to change permissions on
    +     * @throws IOException on any error
    +     */
    +    public void restrictDirectoryPermissions(String dir) throws IOException {
    +        Set<PosixFilePermission> perms = new HashSet<>(
    +                Arrays.asList(PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE,
    +                        PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ,
    +                        PosixFilePermission.GROUP_EXECUTE));
    +        Files.setPosixFilePermissions(FileSystems.getDefault().getPath(dir), perms);
    +    }
    +
    +    /**
    +     * Move fromDir to toDir, and try to make it an atomic move if possible
    +     * @param fromDir what to move
    +     * @param toDir where to move it from
    +     * @throws IOException on any error
    +     */
    +    public void moveDriectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +        FileUtils.forceMkdir(toDir);
    +        Files.move(fromDir.toPath(), toDir.toPath(), StandardCopyOption.ATOMIC_MOVE);
    +    }
    +    
    +    /**
    +     * @return true if an atomic directory move works, else false.
    +     */
    +    public boolean supportsAtomicDirectoryMove() {
    +        return true;
    +    }
    +    
    +    /**
    +     * Setup permissions properly for an internal blob store path
    +     * @param path the path to the permissions
    --- End diff --
    
    path to blobstor?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78963666
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,328 @@
    +/**
    + * 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.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 java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +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.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this.superConf = supervisor.getConf();
    +        this.stormClusterState = supervisor.getStormClusterState();
    +        this.syncSupEventManager = supervisor.getEventManger();
    +        this.assignmentVersions = new AtomicReference<>(new HashMap<>());
    +        this.assignmentId = supervisor.getAssignmentId();
    +        this.iSuper = supervisor.getiSupervisor();
    +        this.localizer = supervisor.getAsyncLocalizer();
    +        this.host = supervisor.getHostName();
    +        this.localState = supervisor.getLocalState();
    +        this.clusterState = supervisor.getStormClusterState();
    +        this.cachedAssignments = supervisor.getCurrAssignment();
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, supervisor.getSharedContext());
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +        
    +        try {
    +            Collection<String> workers = SupervisorUtils.supervisorWorkerIds(superConf);
    +            for (Slot slot: slots.values()) {
    +                String workerId = slot.getWorkerId();
    +                if (workerId != null) {
    +                    workers.remove(workerId);
    +                }
    +            }
    +            if (!workers.isEmpty()) {
    +                supervisor.killWorkers(workers, launcher);
    +            }
    +        } catch (Exception e) {
    +            LOG.warn("Error trying to clean up old workers", e);
    +        }
    +
    +        //All the slots/assignments should be recovered now, so we can clean up anything that we don't expect to be here
    +        try {
    +            localizer.cleanupUnusedTopologies();
    +        } catch (Exception e) {
    +            LOG.warn("Error trying to clean up old topologies", e);
    +        }
    +        
    +        for (Slot slot: slots.values()) {
    +            slot.start();
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        return new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    +                            if (actions == null) {
    +                                actions = new HashSet<>();
    +                                filtered.put(port.intValue(), actions);
    +                            }
    +                            actions.add(new TopoProfileAction(topoId, req));
    +                        }
    +                    }
    +                }
    +            }
    +            
    +            for (Integer port: allPorts) {
    +                Slot slot = slots.get(port);
    +                if (slot == null) {
    +                    slot = mkSlot(port);
    +                    slots.put(port, slot);
    +                    slot.start();
    +                }
    +                slot.setNewAssignment(allAssignments.get(port));
    +                slot.addProfilerActions(filtered.get(port));
    +            }
    +            
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +    }
    +    
    +    protected Map<String, VersionedData<Assignment>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +            Map<String, VersionedData<Assignment>> localAssignmentVersion, Runnable callback) throws Exception {
    +        Map<String, VersionedData<Assignment>> updateAssignmentVersion = new HashMap<>();
    +        for (String topoId : topoIds) {
    +            Integer recordedVersion = -1;
    +            Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +            VersionedData<Assignment> locAssignment = localAssignmentVersion.get(topoId);
    +            if (locAssignment != null) {
    +                recordedVersion = locAssignment.getVersion();
    +            }
    +            if (version == null) {
    +                // ignore
    +            } else if (version == recordedVersion) {
    +                updateAssignmentVersion.put(topoId, locAssignment);
    +            } else {
    +                VersionedData<Assignment> assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                updateAssignmentVersion.put(topoId, assignmentVersion);
    +            }
    +        }
    +        return updateAssignmentVersion;
    +    }
    +    
    +    protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +        Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +        for (String stormId : stormIds) {
    +            List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +            ret.put(stormId, profileRequests);
    +        }
    +        return ret;
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readAssignments(Map<String, VersionedData<Assignment>> assignmentsSnapshot,
    +            String assignmentId, AtomicInteger retries) {
    +        try {
    +            Map<Integer, LocalAssignment> portLA = new HashMap<Integer, LocalAssignment>();
    +            for (Map.Entry<String, VersionedData<Assignment>> assignEntry : assignmentsSnapshot.entrySet()) {
    +                String topoId = assignEntry.getKey();
    +                Assignment assignment = assignEntry.getValue().getData();
    +
    +                Map<Integer, LocalAssignment> portTasks = readMyExecutors(topoId, assignmentId, assignment);
    +
    +                for (Map.Entry<Integer, LocalAssignment> entry : portTasks.entrySet()) {
    +
    +                    Integer port = entry.getKey();
    +
    +                    LocalAssignment la = entry.getValue();
    +
    +                    if (!portLA.containsKey(port)) {
    +                        portLA.put(port, la);
    +                    } else {
    +                        throw new RuntimeException("Should not have multiple topologies assigned to one port "
    +                          + port + " " + la + " " + portLA);
    +                    }
    +                }
    +            }
    +            retries.set(0);
    +            return portLA;
    +        } catch (RuntimeException e) {
    +            if (retries.get() > 2) {
    +                throw e;
    +            } else {
    +                retries.addAndGet(1);
    +            }
    +            LOG.warn("{} : retrying {} of 3", e.getMessage(), retries.get());
    +            return null;
    +        }
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readMyExecutors(String stormId, String assignmentId, Assignment assignment) {
    +        Map<Integer, LocalAssignment> portTasks = new HashMap<>();
    +        Map<Long, WorkerResources> slotsResources = new HashMap<>();
    +        Map<NodeInfo, WorkerResources> nodeInfoWorkerResourcesMap = assignment.get_worker_resources();
    +        if (nodeInfoWorkerResourcesMap != null) {
    +            for (Map.Entry<NodeInfo, WorkerResources> entry : nodeInfoWorkerResourcesMap.entrySet()) {
    +                if (entry.getKey().get_node().equals(assignmentId)) {
    +                    Set<Long> ports = entry.getKey().get_port();
    +                    for (Long port : ports) {
    +                        slotsResources.put(port, entry.getValue());
    +                    }
    +                }
    +            }
    +        }
    +        Map<List<Long>, NodeInfo> executorNodePort = assignment.get_executor_node_port();
    +        if (executorNodePort != null) {
    +            for (Map.Entry<List<Long>, NodeInfo> entry : executorNodePort.entrySet()) {
    +                if (entry.getValue().get_node().equals(assignmentId)) {
    +                    for (Long port : entry.getValue().get_port()) {
    +                        LocalAssignment localAssignment = portTasks.get(port.intValue());
    +                        if (localAssignment == null) {
    +                            List<ExecutorInfo> executors = new ArrayList<ExecutorInfo>();
    --- End diff --
    
    Nitpick: \<ExecutorInfo\> on the right hand side


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77527879
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,420 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + * TODO once we have replaced the original supervisor merge this with
    + * Localizer and optimize them
    + */
    +public class AsyncLocalizer implements ILocalizer, Shutdownable {
    +    /**
    +     * A future that has already completed.
    +     */
    +    private static class AllDoneFuture implements Future<Void> {
    +
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isCancelled() {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isDone() {
    +            return true;
    +        }
    +
    +        @Override
    +        public Void get() {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void get(long timeout, TimeUnit unit) {
    +            return null;
    +        }
    +
    +    }
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizer.class);
    +
    +    private final Localizer _localizer;
    +    private final ExecutorService _execService;
    +    private final boolean _isLocalMode;
    +    private final Map<String, Object> _conf;
    +    private final Map<String, LocalDownloadedResource> _basicPending;
    +    private final Map<String, LocalDownloadedResource> _blobPending;
    +    private final AdvancedFSOps _fsOps;
    +
    +    private class DownloadBaseBlobsDistributed implements Callable<Void> {
    +        private final String _topologyId;
    +        
    +        public DownloadBaseBlobsDistributed(String topologyId) {
    +            this._topologyId = topologyId;
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            File sr = new File(stormroot);
    +            if (sr.exists()) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    Utils.forceDelete(stormroot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    --- End diff --
    
    I refactored this instead so the code is shared by a single class.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77141710
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/ResourceIsolationInterface.java ---
    @@ -56,4 +64,13 @@
          */
         List<String> getLaunchCommandPrefix(String workerId);
     
    +    /**
    +     * Get the list of PIDs currently in an isolated container
    +     * @param workerId the id of the worker to get these for
    +     * @return the set of PIDs, this will be combined with
    +     * other ways of getting PIDs. An Empty set or null if
    +     * no PIDs are found.
    +     * @throws IOException on any error
    +     */
    --- End diff --
    
    I agree with @d2r, it's just as easy for the implementation to return Collections.emptySet() as null, and null checking can be easy to forget when the code is modified later.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77726704
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Supervisor.java ---
    @@ -17,135 +17,231 @@
      */
     package org.apache.storm.daemon.supervisor;
     
    +import java.io.File;
    +import java.io.IOException;
    +import java.net.UnknownHostException;
    +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 java.util.concurrent.Callable;
    +import java.util.concurrent.atomic.AtomicReference;
    +
     import org.apache.commons.io.FileUtils;
     import org.apache.storm.Config;
     import org.apache.storm.StormTimer;
    -import org.apache.storm.daemon.supervisor.timer.RunProfilerActions;
    +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.supervisor.timer.SupervisorHealthCheck;
     import org.apache.storm.daemon.supervisor.timer.SupervisorHeartbeat;
     import org.apache.storm.daemon.supervisor.timer.UpdateBlobs;
    +import org.apache.storm.event.EventManager;
     import org.apache.storm.event.EventManagerImp;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.localizer.AsyncLocalizer;
    +import org.apache.storm.localizer.ILocalizer;
     import org.apache.storm.localizer.Localizer;
     import org.apache.storm.messaging.IContext;
     import org.apache.storm.metric.StormMetricsRegistry;
     import org.apache.storm.scheduler.ISupervisor;
     import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
     import org.apache.storm.utils.Utils;
     import org.apache.storm.utils.VersionInfo;
    +import org.apache.zookeeper.data.ACL;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -import java.io.File;
    -import java.io.InterruptedIOException;
    -import java.util.Collection;
    -import java.util.Map;
    -import java.util.Set;
    -import java.util.concurrent.Callable;
    -
    -public class Supervisor {
    +public class Supervisor implements DaemonCommon, AutoCloseable {
         private static final Logger LOG = LoggerFactory.getLogger(Supervisor.class);
    +    private final Map<String, Object> conf;
    +    private final IContext sharedContext;
    +    private volatile boolean active;
    +    private final ISupervisor iSupervisor;
    +    private final Utils.UptimeComputer upTime;
    +    private final String stormVersion;
    +    private final IStormClusterState stormClusterState;
    +    private final LocalState localState;
    +    private final String supervisorId;
    +    private final String assignmentId;
    +    private final String hostName;
    +    // used for reporting used ports when heartbeating
    +    private final AtomicReference<Map<Long, LocalAssignment>> currAssignment;
    --- End diff --
    
    Actually it is just on restart of the supervisor.  It recovers the running topology but doesn't update it.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77294585
  
    --- Diff: storm-core/src/jvm/org/apache/storm/utils/ConfigUtils.java ---
    @@ -353,25 +350,21 @@ public LocalState nimbusTopoHistoryStateImpl(Map conf) throws IOException {
         }
     
         // we use this "weird" wrapper pattern temporarily for mocking in clojure test
    -    public static Map readSupervisorStormConf(Map conf, String stormId) throws IOException {
    +    public static Map<String, Object> readSupervisorStormConf(Map<String, Object> conf, String stormId) throws IOException {
             return _instance.readSupervisorStormConfImpl(conf, stormId);
         }
     
    -    public Map readSupervisorStormConfImpl(Map conf, String stormId) throws IOException {
    +    public Map<String, Object> readSupervisorStormConfImpl(Map<String, Object> conf, String stormId) throws IOException {
             String stormRoot = supervisorStormDistRoot(conf, stormId);
             String confPath = supervisorStormConfPath(stormRoot);
             return readSupervisorStormConfGivenPath(conf, confPath);
         }
     
         // we use this "weird" wrapper pattern temporarily for mocking in clojure test
    -    public static StormTopology readSupervisorTopology(Map conf, String stormId) throws IOException {
    -        return _instance.readSupervisorTopologyImpl(conf, stormId);
    -    }
    -
    -    public StormTopology readSupervisorTopologyImpl(Map conf, String stormId) throws IOException {
    +    public static StormTopology readSupervisorTopology(Map conf, String stormId, AdvancedFSOps ops) throws IOException {
    --- End diff --
    
    Don't we need to consider mock like `readSupervisorStormConf()`? If we don't need to consider, let's remove above comment to make consistent.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77154288
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,493 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.io.Reader;
    +import java.io.Writer;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected final Map<String, Object> _topoConf;
    +    protected String _workerId;
    +    protected final String _topologyId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    //Exposed for testing
    +    protected Container(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        assert((assignment == null && port <= 0) ||
    +                (assignment != null && port > 0));
    +        assert(conf != null);
    +        assert(ops != null);
    +        assert(supervisorId != null);
    +        
    +        _port = port;
    +        _ops = ops;
    +        _assignment = assignment;
    +        if (assignment != null) {
    +            _topologyId = assignment.get_topology_id();
    +        } else {
    +            _topologyId = null;
    +        }
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        if (topoConf == null) {
    +            _topoConf = readTopoConf();
    +        } else {
    +            _topoConf = topoConf;
    +        }
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return this.getClass().getSimpleName() + " topo:" + _topologyId + " worker:" + _workerId;
    +    }
    +    
    +    protected Map<String, Object> readTopoConf() throws IOException {
    +        assert(_topologyId != null);
    +        return ConfigUtils.readSupervisorStormConf(_conf, _topologyId);
    +    }
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), port, assignment, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     * @throws IOException on any error
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), -1, null, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.trace("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    --- End diff --
    
    On my Windows 10 install, tasklist only outputs these fields: "Image Name, PID, Session Name, Session#, Mem Usage", but there's a filter to get only processes started by a user. For example: 
    `C:\Users\Stig>tasklist /nh /fi "pid eq 10300" /fi "username eq Stig"`
    outputs
    `bash.exe                     10300 Console                   21      1.484 K`


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76611455
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,437 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = -1;
    +        _assignment = null;
    +        _workerId = workerId;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq"+pid);
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            if (in.readLine() != null) {
    +                ret = true;
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    private boolean isPosixProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("ps", "-o", "user", "-p", String.valueOf(pid));
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            String first = in.readLine();
    +            assert("USER".equals(first));
    +            String processUser;
    +            while ((processUser = in.readLine()) != null) {
    +                if (user.equals(processUser)) {
    +                    ret = true;
    +                    break;
    +                } else {
    +                    LOG.info("Found {} running as {}, but expected it to be {}", pid, processUser, user);
    +                }
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    @Override
    +    public boolean areAllProcessesDead() throws IOException {
    +        Set<Long> pids = getAllPids();
    +        String user = getWorkerUser();
    +        
    +        boolean allDead = true;
    +        for (Long pid: pids) {
    +            if (!isProcessAlive(pid, user)) {
    +                LOG.warn("{}: PID {} is dead", _workerId, pid);
    +            } else {
    +                allDead = false;
    +                break;
    +            }
    +        }
    +        return allDead;
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +    }
    +
    +    /**
    +     * Setup the container to run.  By default this creates the needed directories/links in the
    +     * local file system
    +     * PREREQUISITE: All needed blobs and topology, jars/configs have been downloaded and
    +     * placed in the appropriate locations
    +     * @throws IOException on any error
    +     */
    +    protected void setup() throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    +        }
    +        final String topologyId = _assignment.get_topology_id();
    +        if (!SupervisorUtils.doRequiredTopoFilesExist(_conf, topologyId)) {
    +            LOG.info("Missing topology storm code, so can't launch  worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                    _supervisorId, _port, _workerId);
    +            throw new IllegalStateException("Not all needed files are here!!!!");
    +        }
    +        String pidsPath = ConfigUtils.workerPidsRoot(_conf, _workerId);
    +        String hbPath = ConfigUtils.workerHeartbeatsRoot(_conf, _workerId);
    +    
    +        FileUtils.forceMkdir(new File(pidsPath));
    +        FileUtils.forceMkdir(new File(ConfigUtils.workerTmpRoot(_conf, _workerId)));
    +        FileUtils.forceMkdir(new File(hbPath));
    +    
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        String user = (String) topologyConf.get(Config.TOPOLOGY_SUBMITTER_USER);
    +        writeLogMetadata(topologyConf, user, topologyId);
    +        ConfigUtils.setWorkerUserWSE(_conf, _workerId, user);
    +        createArtifactsLink(topologyId);
    +    
    +        createBlobstoreLinks(topologyId);
    +    }
    +    
    +    /**
    +     * Write out the file used by the log viewer to allow/reject log access
    +     * @param topologyConf the config for the topology
    +     * @param user the user this is going to run as
    +     * @param topologyId the id of the topology
    +     * @throws IOException on any error
    +     */
    +    @SuppressWarnings("unchecked")
    +    protected void writeLogMetadata(Map<String, Object> topologyConf, String user, String topologyId) throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    +        }
    +        Map<String, Object> data = new HashMap<>();
    +        data.put(Config.TOPOLOGY_SUBMITTER_USER, user);
    +        data.put("worker-id", _workerId);
    +
    +        Set<String> logsGroups = new HashSet<>();
    +        //for supervisor-test
    --- End diff --
    
    Copy and paste really don't know


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77421128
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/RunAsUserContainerLauncher.java ---
    @@ -0,0 +1,71 @@
    +/**
    + * 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.Map;
    +
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.utils.LocalState;
    +
    +public class RunAsUserContainerLauncher extends ContainerLauncher {
    +    private final Map<String, Object> _conf;
    +    private final String _supervisorId;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    public RunAsUserContainerLauncher(Map<String, Object> conf, String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +    }
    +
    +    @Override
    +    public Container launchContainer(int port, LocalAssignment assignment, LocalState state) throws IOException {
    +        Container container = new RunAsUserContainer(port, assignment, _conf, _supervisorId, state,
    +                _resourceIsolationManager, false);
    +        container.setup();
    +        container.launch();
    +        return container;
    +    }
    +
    +    @Override
    +    public Container recoverContainer(int port, LocalAssignment assignment, LocalState state) throws IOException {
    +        Container container = null;
    +        try {
    +            container = new RunAsUserContainer(port, assignment, _conf, _supervisorId, state, 
    +                    _resourceIsolationManager, true);
    +        } catch (ContainerRecoveryException e) {
    +            // We could not recover return null
    +        }
    +        return container;
    +    }
    +    
    +    @Override
    +    public Killable recoverContainer(String workerId) throws IOException {
    +        Container container = null;
    +        try {
    +            container = new RunAsUserContainer(workerId, _conf, _supervisorId, 
    +                    _resourceIsolationManager);
    +        } catch (ContainerRecoveryException e) {
    +            // We could not recover return null
    +        }
    +        return container;
    +    }
    --- End diff --
    
    Yes, I botched my comment:  I wanted to suggest recoverContainer(p,a,s) could just call recoverContainer(w), but the two have different return types. So we should leave it.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76475950
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,437 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = -1;
    +        _assignment = null;
    +        _workerId = workerId;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq"+pid);
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            if (in.readLine() != null) {
    +                ret = true;
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    private boolean isPosixProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("ps", "-o", "user", "-p", String.valueOf(pid));
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            String first = in.readLine();
    +            assert("USER".equals(first));
    +            String processUser;
    +            while ((processUser = in.readLine()) != null) {
    +                if (user.equals(processUser)) {
    +                    ret = true;
    +                    break;
    +                } else {
    +                    LOG.info("Found {} running as {}, but expected it to be {}", pid, processUser, user);
    +                }
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    @Override
    +    public boolean areAllProcessesDead() throws IOException {
    +        Set<Long> pids = getAllPids();
    +        String user = getWorkerUser();
    +        
    +        boolean allDead = true;
    +        for (Long pid: pids) {
    +            if (!isProcessAlive(pid, user)) {
    +                LOG.warn("{}: PID {} is dead", _workerId, pid);
    +            } else {
    +                allDead = false;
    +                break;
    +            }
    +        }
    +        return allDead;
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +    }
    +
    +    /**
    +     * Setup the container to run.  By default this creates the needed directories/links in the
    +     * local file system
    +     * PREREQUISITE: All needed blobs and topology, jars/configs have been downloaded and
    +     * placed in the appropriate locations
    +     * @throws IOException on any error
    +     */
    +    protected void setup() throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    +        }
    +        final String topologyId = _assignment.get_topology_id();
    +        if (!SupervisorUtils.doRequiredTopoFilesExist(_conf, topologyId)) {
    +            LOG.info("Missing topology storm code, so can't launch  worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                    _supervisorId, _port, _workerId);
    +            throw new IllegalStateException("Not all needed files are here!!!!");
    +        }
    +        String pidsPath = ConfigUtils.workerPidsRoot(_conf, _workerId);
    +        String hbPath = ConfigUtils.workerHeartbeatsRoot(_conf, _workerId);
    +    
    +        FileUtils.forceMkdir(new File(pidsPath));
    +        FileUtils.forceMkdir(new File(ConfigUtils.workerTmpRoot(_conf, _workerId)));
    +        FileUtils.forceMkdir(new File(hbPath));
    +    
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        String user = (String) topologyConf.get(Config.TOPOLOGY_SUBMITTER_USER);
    +        writeLogMetadata(topologyConf, user, topologyId);
    +        ConfigUtils.setWorkerUserWSE(_conf, _workerId, user);
    +        createArtifactsLink(topologyId);
    +    
    +        createBlobstoreLinks(topologyId);
    +    }
    +    
    +    /**
    +     * Write out the file used by the log viewer to allow/reject log access
    +     * @param topologyConf the config for the topology
    +     * @param user the user this is going to run as
    +     * @param topologyId the id of the topology
    +     * @throws IOException on any error
    +     */
    +    @SuppressWarnings("unchecked")
    +    protected void writeLogMetadata(Map<String, Object> topologyConf, String user, String topologyId) throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    +        }
    +        Map<String, Object> data = new HashMap<>();
    +        data.put(Config.TOPOLOGY_SUBMITTER_USER, user);
    +        data.put("worker-id", _workerId);
    +
    +        Set<String> logsGroups = new HashSet<>();
    +        //for supervisor-test
    +        if (topologyConf.get(Config.LOGS_GROUPS) != null) {
    +            List<String> groups = (List<String>) topologyConf.get(Config.LOGS_GROUPS);
    +            for (String group : groups){
    +                logsGroups.add(group);
    +            }
    --- End diff --
    
    We could use addAll instead of looping, as we do below, here and elsewhere.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76814090
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,769 @@
    +/**
    + * 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);
    +            }
    +            if (container != null) {
    +                sb.append(" container: ");
    +                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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WAITING_FOR_WORKER_START) {
    +                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 equivilant(LocalAssignment a, LocalAssignment b) {
    --- End diff --
    
    nit, equivalent?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78941512
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,335 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.FileOutputStream;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.io.Writer;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    +    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
    +    
    +    /**
    +     * Factory to create a new AdvancedFSOps
    +     * @param conf the configuration of the process
    +     * @return the appropriate instance of the class for this config and environment.
    +     */
    +    public static AdvancedFSOps make(Map<String, Object> conf) {
    +        if (Utils.isOnWindows()) {
    +            return new AdvancedWindowsFSOps(conf);
    +        }
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new AdvancedRunAsUserFSOps(conf);
    +        }
    +        return new AdvancedFSOps();
    +    }
    +    
    +    private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
    +        private final Map<String, Object> _conf;
    +        
    +        public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
    +            if (Utils.isOnWindows()) {
    +                throw new UnsupportedOperationException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +            _conf = conf;
    +        }
    +        
    +        @Override
    +        public void setupBlobPermissions(File path, String user) throws IOException {
    +            String logPrefix = "setup blob permissions for " + path;
    +            SupervisorUtils.processLauncherAndWait(_conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
    +        }
    +        
    +        @Override
    +        public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +            String absolutePath = path.getAbsolutePath();
    +            LOG.info("Deleting path {}", absolutePath);
    +            if (user == null) {
    +                user = Files.getOwner(path.toPath()).getName();
    +            }
    +            List<String> commands = new ArrayList<>();
    +            commands.add("rmr");
    +            commands.add(absolutePath);
    +            SupervisorUtils.processLauncherAndWait(_conf, user, commands, null, logPrefix);
    +            if (Utils.checkFileExists(absolutePath)) {
    +                throw new RuntimeException(path + " was not deleted.");
    +            }
    +        }
    +        
    +        @Override
    +        public void setupStormCodeDir(Map<String, Object> topologyConf, File path) throws IOException {
    +            SupervisorUtils.setupStormCodeDir(_conf, topologyConf, path.getCanonicalPath());
    +        }
    +    }
    +    
    +    /**
    +     * Operations that need to override the default ones when running on Windows
    +     *
    +     */
    +    private static class AdvancedWindowsFSOps extends AdvancedFSOps {
    +
    +        public AdvancedWindowsFSOps(Map<String, Object> conf) {
    +            if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +        }
    +        
    +        @Override
    +        public void restrictDirectoryPermissions(File dir) throws IOException {
    +            //NOOP, if windows gets support for run as user we will need to find a way to suppor this
    --- End diff --
    
    Nitpick: There's a t missing in support


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76610605
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,429 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.AdvancedFSOps;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = -1;
    +        _assignment = null;
    +        _workerId = workerId;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    --- End diff --
    
    done


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76805459
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,493 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.io.Reader;
    +import java.io.Writer;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected final Map<String, Object> _topoConf;
    +    protected String _workerId;
    +    protected final String _topologyId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    //Exposed for testing
    +    protected Container(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        assert((assignment == null && port <= 0) ||
    +                (assignment != null && port > 0));
    +        assert(conf != null);
    +        assert(ops != null);
    +        assert(supervisorId != null);
    +        
    +        _port = port;
    +        _ops = ops;
    +        _assignment = assignment;
    +        if (assignment != null) {
    +            _topologyId = assignment.get_topology_id();
    +        } else {
    +            _topologyId = null;
    +        }
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        if (topoConf == null) {
    +            _topoConf = readTopoConf();
    +        } else {
    +            _topoConf = topoConf;
    +        }
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return this.getClass().getSimpleName() + " topo:" + _topologyId + " worker:" + _workerId;
    +    }
    +    
    +    protected Map<String, Object> readTopoConf() throws IOException {
    +        assert(_topologyId != null);
    +        return ConfigUtils.readSupervisorStormConf(_conf, _topologyId);
    +    }
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), port, assignment, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     * @throws IOException on any error
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), -1, null, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.trace("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    --- End diff --
    
    should this check the pids user, just like we check in the posix case?


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    I found another issue: 
    When I rebalance 3 workers into 1 worker, all workers are killed first (expected) and AsyncLocalizer clear out topology codes since all workers are killed. 
    But Supervisor doesn't download topology code again while starting new worker, so it goes wrong and worker and/or supervisor are killed.
    
    This seems to be a kind of race condition (Slot and AsyncLocalizer) and I saw two scenarios: 
    
    1. Worker can be launched but topology directory is removed after launching so worker is crashed. Slot tries to relaunch worker but throws IllegalStateException because topology directory is gone and supervisor also be killed.
    
    2. Supervisor is killed even before launching worker.
    
    After this, Supervisor will consistently be killed unless clearing out supervisor directory as same as above comment.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @srdo on the  `newAssignment.compareAndSet` you are 100% correct.  I originally had it in there to clear out the assignment when it is processed, but then realized that each time a new assignment is downloaded it would go back to what it was before and we needed to handle that. I'll clean it up.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77418598
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,420 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + * TODO once we have replaced the original supervisor merge this with
    + * Localizer and optimize them
    + */
    +public class AsyncLocalizer implements ILocalizer, Shutdownable {
    +    /**
    +     * A future that has already completed.
    +     */
    +    private static class AllDoneFuture implements Future<Void> {
    +
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isCancelled() {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isDone() {
    +            return true;
    +        }
    +
    +        @Override
    +        public Void get() {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void get(long timeout, TimeUnit unit) {
    +            return null;
    +        }
    +
    +    }
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizer.class);
    +
    +    private final Localizer _localizer;
    +    private final ExecutorService _execService;
    +    private final boolean _isLocalMode;
    +    private final Map<String, Object> _conf;
    +    private final Map<String, LocalDownloadedResource> _basicPending;
    +    private final Map<String, LocalDownloadedResource> _blobPending;
    +    private final AdvancedFSOps _fsOps;
    +
    +    private class DownloadBaseBlobsDistributed implements Callable<Void> {
    +        private final String _topologyId;
    +        
    +        public DownloadBaseBlobsDistributed(String topologyId) {
    +            this._topologyId = topologyId;
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            File sr = new File(stormroot);
    +            if (sr.exists()) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    Utils.forceDelete(stormroot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            try {
    +                String stormJarKey = ConfigUtils.masterStormJarKey(_topologyId);
    +                String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +                String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +                String jarPath = ConfigUtils.supervisorStormJarPath(tmproot);
    +                String codePath = ConfigUtils.supervisorStormCodePath(tmproot);
    +                String confPath = ConfigUtils.supervisorStormConfPath(tmproot);
    +                FileUtils.forceMkdir(new File(tmproot));
    +                _fsOps.restrictDirectoryPermissions(tmproot);
    +                ClientBlobStore blobStore = Utils.getClientBlobStoreForSupervisor(_conf);
    +                try {
    +                    Utils.downloadResourcesAsSupervisor(stormJarKey, jarPath, blobStore);
    +                    Utils.downloadResourcesAsSupervisor(stormCodeKey, codePath, blobStore);
    +                    Utils.downloadResourcesAsSupervisor(stormConfKey, confPath, blobStore);
    +                } finally {
    +                    blobStore.shutdown();
    +                }
    +                Utils.extractDirFromJar(jarPath, ConfigUtils.RESOURCES_SUBDIR, tmproot);
    +                _fsOps.moveDirectoryPreferAtomic(new File(tmproot), new File(stormroot));
    +                SupervisorUtils.setupStormCodeDir(_conf, ConfigUtils.readSupervisorStormConf(_conf, _topologyId), stormroot);
    +                deleteAll = false;
    +            } finally {
    +                if (deleteAll) {
    +                    LOG.info("Failed to download basic resources for topology-id {}", _topologyId);
    +                    Utils.forceDelete(tmproot);
    +                    Utils.forceDelete(stormroot);
    +                }
    +            }
    +            return null;
    +        }
    +    }
    +    
    +    private class DownloadBaseBlobsLocal implements Callable<Void> {
    +        private final String _topologyId;
    +        
    +        public DownloadBaseBlobsLocal(String topologyId) {
    +            this._topologyId = topologyId;
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            File sr = new File(stormroot);
    +            if (sr.exists()) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    Utils.forceDelete(stormroot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            try {
    +                BlobStore blobStore = Utils.getNimbusBlobStore(_conf, null, null);
    +                FileOutputStream codeOutStream = null;
    +                FileOutputStream confOutStream = null;
    +                try {
    +                    FileUtils.forceMkdir(new File(tmproot));
    +                    String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +                    String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +                    String codePath = ConfigUtils.supervisorStormCodePath(tmproot);
    +                    String confPath = ConfigUtils.supervisorStormConfPath(tmproot);
    +                    codeOutStream = new FileOutputStream(codePath);
    +                    blobStore.readBlobTo(stormCodeKey, codeOutStream, null);
    +                    confOutStream = new FileOutputStream(confPath);
    +                    blobStore.readBlobTo(stormConfKey, confOutStream, null);
    +                } finally {
    +                    if (codeOutStream != null)
    +                        codeOutStream.close();
    +                    if (confOutStream != null)
    +                        codeOutStream.close();
    --- End diff --
    
    * braces
    * catch IOException


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @harshach it has been 6 hours so I am going to check this in because I don't want to have to try and keep it up to date as the code base changes.  If you do find anything as time goes on I will be very happy to address your concerns in other pull requests.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    I merged this into master and squashed the commits so I am closing the pull request, but will leave the JIRA open so we can look into pulling it back into 1.x as well.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77532577
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/LocalDownloadedResource.java ---
    @@ -0,0 +1,107 @@
    +/**
    + * 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.localizer;
    +
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class LocalDownloadedResource {
    +    private static final Logger LOG = LoggerFactory.getLogger(LocalDownloadedResource.class);
    +    private static class NoCancelFuture<T> implements Future<T> {
    +        private final Future<T> _wrapped;
    +        
    +        public NoCancelFuture(Future<T> wrapped) {
    +            _wrapped = wrapped;
    +        }
    +        
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            //cancel not currently supported
    --- End diff --
    
    OK. Thanks


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77294478
  
    --- Diff: storm-core/src/jvm/org/apache/storm/utils/ConfigUtils.java ---
    @@ -512,6 +463,10 @@ public static String workerTmpRoot(Map conf, String id) {
         public static String workerPidPath(Map conf, String id, String pid) {
             return (workerPidsRoot(conf, id) + FILE_SEPARATOR + pid);
         }
    +    
    +    public static String workerPidPath(Map<String, Object> conf, String id, long pid) {
    +        return (workerPidsRoot(conf, id) + FILE_SEPARATOR + pid);
    --- End diff --
    
    Nit: might be better to call workerPidPath(conf, id, String.valueOf(pid)) or vice versa.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76245649
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,417 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.AdvancedFSOps;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill the processes in this container nicely.
    +     * kill -15 equivalent
    +     * @throws IOException on any error
    +     */
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Kill the processes in this container violently.
    +     * kill -9 equivalent
    +     * @throws IOException on any error
    +     */
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq"+pid);
    +            pb.redirectError(Redirect.INHERIT);
    +            Process p = pb.start();
    +            try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +                //TODO would be really nice to parse out the user
    +                if (in.readLine() != null) {
    +                    return true;
    +                }
    +            }
    +            return false;
    +        }
    +        //Not on Windows assume POSIX
    +        ProcessBuilder pb = new ProcessBuilder("ps", "-o", "user", "-p", String.valueOf(pid));
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            String first = in.readLine();
    +            assert("USER".equals(first));
    +            String processUser;
    +            while ((processUser = in.readLine()) != null) {
    +                if (user.equals(processUser)) {
    +                    return true;
    +                } else {
    +                    LOG.info("Found {} running as {}, but expected it to be {}", pid, processUser, user);
    +                }
    +            }
    +        }
    +        return false;
    +    }
    +    
    +    /**
    +     * @return true if all of the processes are dead, else false
    +     * @throws IOException on any error
    +     */
    +    public boolean areAllProcessesDead() throws IOException {
    +        Set<Long> pids = getAllPids();
    +        String user = getWorkerUser();
    +        
    +        boolean allDead = true;
    +        for (Long pid: pids) {
    +            if (!isProcessAlive(pid, user)) {
    +                LOG.warn("{}: PID {} is dead", _workerId, pid);
    +            } else {
    +                allDead = false;
    +                break;
    +            }
    +        }
    +        return allDead;
    +    }
    +
    +    /**
    +     * Clean up the container. It is not coming back.
    +     * by default do the same thing as when restarting.
    +     * @throws IOException on any error
    +     */
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +    }
    +
    +    /**
    +     * Setup the container to run.  By default this creates the needed directories/links in the
    +     * local file system
    +     * PREREQUISITE: All needed blobs and topology, jars/configs have been downloaded and
    +     * placed in the appropriate locations
    +     * @throws IOException on any error
    +     */
    +    public void setup() throws IOException {
    +        final String topologyId = _assignment.get_topology_id();
    +        if (!SupervisorUtils.doRequiredTopoFilesExist(_conf, topologyId)) {
    +            LOG.info("Missing topology storm code, so can't launch  worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                    _supervisorId, _port, _workerId);
    +            throw new IllegalStateException("Not all needed files are here!!!!");
    +        }
    +        String pidsPath = ConfigUtils.workerPidsRoot(_conf, _workerId);
    +        String hbPath = ConfigUtils.workerHeartbeatsRoot(_conf, _workerId);
    +    
    +        FileUtils.forceMkdir(new File(pidsPath));
    +        FileUtils.forceMkdir(new File(ConfigUtils.workerTmpRoot(_conf, _workerId)));
    +        FileUtils.forceMkdir(new File(hbPath));
    +    
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        String user = (String) topologyConf.get(Config.TOPOLOGY_SUBMITTER_USER);
    +        writeLogMetadata(topologyConf, user, topologyId);
    +        ConfigUtils.setWorkerUserWSE(_conf, _workerId, user);
    +        createArtifactsLink(topologyId);
    +    
    +        createBlobstoreLinks(topologyId);
    +    }
    +    
    +    /**
    +     * Write out the file used by the log viewer to allow/reject log access
    +     * @param topologyConf the config for the topology
    +     * @param user the user this is going to run as
    +     * @param topologyId the id of the topology
    +     * @throws IOException on any error
    +     */
    +    @SuppressWarnings("unchecked")
    +    protected void writeLogMetadata(Map<String, Object> topologyConf, String user, String topologyId) throws IOException {
    +        Map<String, Object> data = new HashMap<>();
    +        data.put(Config.TOPOLOGY_SUBMITTER_USER, user);
    +        data.put("worker-id", _workerId);
    +
    +        Set<String> logsGroups = new HashSet<>();
    +        //for supervisor-test
    +        if (topologyConf.get(Config.LOGS_GROUPS) != null) {
    +            List<String> groups = (List<String>) topologyConf.get(Config.LOGS_GROUPS);
    +            for (String group : groups){
    +                logsGroups.add(group);
    +            }
    +        }
    +        if (topologyConf.get(Config.TOPOLOGY_GROUPS) != null) {
    +            List<String> topGroups = (List<String>) topologyConf.get(Config.TOPOLOGY_GROUPS);
    +            logsGroups.addAll(topGroups);
    +        }
    +        data.put(Config.LOGS_GROUPS, logsGroups.toArray());
    +
    +        Set<String> logsUsers = new HashSet<>();
    +        if (topologyConf.get(Config.LOGS_USERS) != null) {
    +            List<String> logUsers = (List<String>) topologyConf.get(Config.LOGS_USERS);
    +            for (String logUser : logUsers){
    +                logsUsers.add(logUser);
    +            }
    +        }
    +        if (topologyConf.get(Config.TOPOLOGY_USERS) != null) {
    +            List<String> topUsers = (List<String>) topologyConf.get(Config.TOPOLOGY_USERS);
    +            for (String logUser : topUsers){
    +                logsUsers.add(logUser);
    +            }
    +        }
    +        data.put(Config.LOGS_USERS, logsUsers.toArray());
    +
    +        File file = ConfigUtils.getLogMetaDataFile(_conf, topologyId, _port);
    +        File parent = file.getParentFile();
    +        if (!Utils.checkFileExists(parent)) {
    +            FileUtils.forceMkdir(file.getParentFile());
    +            _ops.setupStormCodeDir(ConfigUtils.readSupervisorStormConf(_conf, topologyId), file.getParentFile().getCanonicalPath());
    +        }
    +        Yaml yaml = new Yaml();
    +        try (FileWriter writer = new FileWriter(file)) {
    +            yaml.dump(data, writer);
    +        }
    +    }
    +    
    +    /**
    +     * Create symlink from the containers directory/artifacts to the artifacts directory
    +     * @param topologyId the id of the topology this is for
    +     * @throws IOException on any error
    +     */
    +    protected void createArtifactsLink(String topologyId) throws IOException {
    +        String workerDir = ConfigUtils.workerRoot(_conf, _workerId);
    +        String topoDir = ConfigUtils.workerArtifactsRoot(_conf, topologyId);
    +        if (Utils.checkFileExists(workerDir)) {
    +            LOG.debug("Creating symlinks for worker-id: {} topology-id: {} to its port artifacts directory", _workerId, topologyId);
    +            Utils.createSymlink(workerDir, topoDir, "artifacts", String.valueOf(_port));
    +        }
    --- End diff --
    
    I agree the code way copied and pasted from before, but we should have eliminated the races so if there worker dir is not there we have a bug and should do something about it.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76609083
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,300 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.Writer;
    +import java.nio.file.FileSystems;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.Paths;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    +    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
    +    
    +    /**
    +     * Factory to create a new AdvancedFSOps
    +     * @param conf the configuration of the process
    +     * @return the appropriate instance of the class for this config and environment.
    +     */
    +    public static AdvancedFSOps mk(Map<String, Object> conf) {
    +        if (Utils.isOnWindows()) {
    +            return new AdvancedWindowsFSOps(conf);
    +        }
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new AdvancedRunAsUserFSOps(conf);
    +        }
    +        return new AdvancedFSOps();
    +    }
    +    
    +    private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
    +        private final Map<String, Object> _conf;
    +        
    +        public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
    +            if (Utils.isOnWindows()) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +            _conf = conf;
    +        }
    +        
    +        @Override
    +        public void setupBlobPermissions(File path, String user) throws IOException {
    +            String logPrefix = "setup blob permissions for " + path;
    +            SupervisorUtils.processLauncherAndWait(_conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
    +        }
    +        
    +        @Override
    +        public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +            String absolutePath = path.getAbsolutePath();
    +            LOG.debug("Deleting path {}", absolutePath);
    +            if (user == null) {
    +                user = Files.getOwner(path.toPath()).getName();
    +            }
    +            List<String> commands = new ArrayList<>();
    +            commands.add("rmr");
    +            commands.add(absolutePath);
    +            SupervisorUtils.processLauncherAndWait(_conf, user, commands, null, logPrefix);
    +            if (Utils.checkFileExists(absolutePath)) {
    +                throw new RuntimeException(path + " was not deleted.");
    +            }
    +        }
    +        
    +        @Override
    +        public void setupStormCodeDir(Map<String, Object> topologyConf, File path) throws IOException {
    +            SupervisorUtils.setupStormCodeDir(_conf, topologyConf, path.getCanonicalPath());
    +        }
    +    }
    +    
    +    /**
    +     * Operations that need to override the default ones when running on Windows
    +     *
    +     */
    +    private static class AdvancedWindowsFSOps extends AdvancedFSOps {
    +
    +        public AdvancedWindowsFSOps(Map<String, Object> conf) {
    +            if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +        }
    +        
    +        @Override
    +        public void restrictDirectoryPermissions(String dir) throws IOException {
    +            //NOOP, if windows gets support for run as user we will need to find a way to suppor this
    +        }
    +        
    +        @Override
    +        public void moveDriectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // This is not atomic but it does work
    +            FileUtils.moveDirectory(fromDir, toDir);
    +        }
    +        
    +        @Override
    +        public boolean supportsAtomicDirectoryMove() {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // FileUtils.moveDirectory is not atomic
    +            return false;
    +        }
    +    }
    +    
    +    
    +    protected AdvancedFSOps() {
    +        //NOOP, but restricted permissions
    +    }
    +
    +    /**
    +     * Set directory permissions to (OWNER)RWX (GROUP)R-X (OTHER)---
    +     * On some systems that do not support this, it may become a noop
    +     * @param dir the directory to change permissions on
    +     * @throws IOException on any error
    +     */
    +    public void restrictDirectoryPermissions(String dir) throws IOException {
    +        Set<PosixFilePermission> perms = new HashSet<>(
    +                Arrays.asList(PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE,
    +                        PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ,
    +                        PosixFilePermission.GROUP_EXECUTE));
    +        Files.setPosixFilePermissions(FileSystems.getDefault().getPath(dir), perms);
    +    }
    +
    +    /**
    +     * Move fromDir to toDir, and try to make it an atomic move if possible
    +     * @param fromDir what to move
    +     * @param toDir where to move it from
    +     * @throws IOException on any error
    +     */
    +    public void moveDriectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +        FileUtils.forceMkdir(toDir);
    +        Files.move(fromDir.toPath(), toDir.toPath(), StandardCopyOption.ATOMIC_MOVE);
    +    }
    +    
    +    /**
    +     * @return true if an atomic directory move works, else false.
    +     */
    +    public boolean supportsAtomicDirectoryMove() {
    +        return true;
    +    }
    +    
    +    /**
    +     * Setup permissions properly for an internal blob store path
    +     * @param path the path to the permissions
    +     * @param user the user to change the permissions for
    +     * @throws IOException on any error
    +     */
    +    public void setupBlobPermissions(File path, String user) throws IOException {
    +        //Normally this is a NOOP
    +    }
    +
    +    /**
    +     * Delete a file or a directory and all of the children. If it exists.
    +     * @param path what to delete
    +     * @param user who to delete it as if doing it as someone else is supported
    +     * @param logPrefix if an external process needs to be launched to delete 
    +     * the object what prefix to include in the logs
    +     * @throws IOException on any error.
    +     */
    +    public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +        LOG.debug("Deleting path {}", path);
    --- End diff --
    
    It does in the runasuserfsops.  There are two inner sub classes.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76535956
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,569 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState localState, ResourceIsolationInterface resourceIsolationManager, boolean recover)
    +            throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException(
    +                    "Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            _localState.setApprovedWorkers(workersToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = null;
    +        try (FileReader reader = new FileReader(str); BufferedReader br = new BufferedReader(reader)) {
    +            workerPid = br.readLine().trim();
    +        }
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        return runProfilingCommand(command, env, logPrefix, targetFile);
    +    }
    +
    +    /**
    +     * Get the command to run when doing profiling
    +     * @param action the profiling action to perform
    +     * @param stop if this is meant to stop the profiling or start it
    +     * @param workerPid the PID of the process to profile
    +     * @param targetDir the current working directory of the worker process
    +     * @return the command to run for profiling.
    +     */
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        if (action == ProfileAction.JMAP_DUMP) {
    --- End diff --
    
    could we make this a switch statement with an if for JPROFILE_STOP?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76812876
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    +                            if (actions == null) {
    +                                actions = new HashSet<>();
    +                                filtered.put(port.intValue(), actions);
    +                            }
    +                            actions.add(new TopoProfileAction(topoId, req));
    +                        }
    +                    }
    +                }
    +            }
    +            
    +            for (Integer port: allPorts) {
    +                Slot slot = slots.get(port);
    +                if (slot == null) {
    +                    slot = mkSlot(port);
    +                    slots.put(port, slot);
    +                }
    +                slot.setNewAssignment(allAssignments.get(port));
    +                slot.addProfilerActions(filtered.get(port));
    +            }
    +            
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +    }
    +    
    +    protected Map<String, VersionedData<Assignment>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +            Map<String, VersionedData<Assignment>> localAssignmentVersion, Runnable callback) throws Exception {
    +        Map<String, VersionedData<Assignment>> updateAssignmentVersion = new HashMap<>();
    +        for (String topoId : topoIds) {
    +            Integer recordedVersion = -1;
    +            Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +            VersionedData<Assignment> locAssignment = localAssignmentVersion.get(topoId);
    +            if (locAssignment != null) {
    +                recordedVersion = locAssignment.getVersion();
    +            }
    +            if (version == null) {
    +                // ignore
    +            } else if (version == recordedVersion) {
    +                updateAssignmentVersion.put(topoId, locAssignment);
    +            } else {
    +                VersionedData<Assignment> assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                updateAssignmentVersion.put(topoId, assignmentVersion);
    +            }
    +        }
    +        return updateAssignmentVersion;
    +    }
    +    
    +    protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +        Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +        for (String stormId : stormIds) {
    +            List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +            ret.put(stormId, profileRequests);
    +        }
    +        return ret;
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readAssignments(Map<String, VersionedData<Assignment>> assignmentsSnapshot,
    +            String assignmentId, AtomicInteger retries) {
    +        try {
    +            Map<Integer, LocalAssignment> portLA = new HashMap<Integer, LocalAssignment>();
    +            for (Map.Entry<String, VersionedData<Assignment>> assignEntry : assignmentsSnapshot.entrySet()) {
    +                String topoId = assignEntry.getKey();
    +                Assignment assignment = assignEntry.getValue().getData();
    +
    +                Map<Integer, LocalAssignment> portTasks = readMyExecutors(topoId, assignmentId, assignment);
    +
    +                for (Map.Entry<Integer, LocalAssignment> entry : portTasks.entrySet()) {
    +
    +                    Integer port = entry.getKey();
    +
    +                    LocalAssignment la = entry.getValue();
    +
    +                    if (!portLA.containsKey(port)) {
    +                        portLA.put(port, la);
    +                    } else {
    +                        throw new RuntimeException("Should not have multiple topologys assigned to one port");
    --- End diff --
    
    nit, topologies


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77000695
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,628 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +
    +    //For testing purposes
    +    public BasicContainer(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager, LocalState localState,
    +            String profileCmd) throws IOException {
    +        super(ops, port, assignment, conf, topoConf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +        _profileCmd = profileCmd;
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState localState, ResourceIsolationInterface resourceIsolationManager, boolean recover)
    +            throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +
    +        String stormHome = System.getProperty("storm.home");
    --- End diff --
    
    Minor: `System.getProperty("storm.home")` might be better to place to Utils or so since there're some other places which uses this. I'm also OK to keep this as it is and address this from other issue.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77357693
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    +                            if (actions == null) {
    +                                actions = new HashSet<>();
    +                                filtered.put(port.intValue(), actions);
    +                            }
    +                            actions.add(new TopoProfileAction(topoId, req));
    +                        }
    +                    }
    +                }
    +            }
    +            
    +            for (Integer port: allPorts) {
    +                Slot slot = slots.get(port);
    +                if (slot == null) {
    +                    slot = mkSlot(port);
    +                    slots.put(port, slot);
    +                }
    +                slot.setNewAssignment(allAssignments.get(port));
    +                slot.addProfilerActions(filtered.get(port));
    +            }
    +            
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +    }
    +    
    +    protected Map<String, VersionedData<Assignment>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +            Map<String, VersionedData<Assignment>> localAssignmentVersion, Runnable callback) throws Exception {
    +        Map<String, VersionedData<Assignment>> updateAssignmentVersion = new HashMap<>();
    +        for (String topoId : topoIds) {
    +            Integer recordedVersion = -1;
    +            Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +            VersionedData<Assignment> locAssignment = localAssignmentVersion.get(topoId);
    +            if (locAssignment != null) {
    +                recordedVersion = locAssignment.getVersion();
    +            }
    +            if (version == null) {
    +                // ignore
    +            } else if (version == recordedVersion) {
    +                updateAssignmentVersion.put(topoId, locAssignment);
    +            } else {
    +                VersionedData<Assignment> assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                updateAssignmentVersion.put(topoId, assignmentVersion);
    +            }
    +        }
    +        return updateAssignmentVersion;
    +    }
    +    
    +    protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +        Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +        for (String stormId : stormIds) {
    +            List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +            ret.put(stormId, profileRequests);
    +        }
    +        return ret;
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readAssignments(Map<String, VersionedData<Assignment>> assignmentsSnapshot,
    +            String assignmentId, AtomicInteger retries) {
    +        try {
    +            Map<Integer, LocalAssignment> portLA = new HashMap<Integer, LocalAssignment>();
    +            for (Map.Entry<String, VersionedData<Assignment>> assignEntry : assignmentsSnapshot.entrySet()) {
    +                String topoId = assignEntry.getKey();
    +                Assignment assignment = assignEntry.getValue().getData();
    +
    +                Map<Integer, LocalAssignment> portTasks = readMyExecutors(topoId, assignmentId, assignment);
    +
    +                for (Map.Entry<Integer, LocalAssignment> entry : portTasks.entrySet()) {
    +
    +                    Integer port = entry.getKey();
    +
    +                    LocalAssignment la = entry.getValue();
    +
    +                    if (!portLA.containsKey(port)) {
    +                        portLA.put(port, la);
    +                    } else {
    +                        throw new RuntimeException("Should not have multiple topologies assigned to one port");
    --- End diff --
    
    Would it be useful to include the topologies and the port in case we want to debug 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    I think I have addressed most of the issues so far.  I have been running some manual tests and have run a cluster with run as user and cgroup enforcement enabled.  I plan on doing some more tests, but I think the current pull request should be ready for anyone who whats to take a look at it/test it to do so.
    
    It should be a drop in replacement for the current supervisor, and I have even tested it doing a rolling upgrade/downgrade.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by ptgoetz <gi...@git.apache.org>.
Github user ptgoetz commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @revans2 I'm also in the process of reviewing this. I hope to finish by the end of the day. So far everything looks good to me.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79027439
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    --- End diff --
    
    Agreed, This was a copy and paste, but you are right it would be cleaner.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    Sorry, git just ate all of my more recent changes, and I have no idea what happened.  I'll try to see if I can recover them, but I might have to start over...


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76271453
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,429 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.AdvancedFSOps;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = -1;
    +        _assignment = null;
    +        _workerId = workerId;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    --- End diff --
    
    should we break this into two methods one for windows and other for linux


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76274105
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Supervisor.java ---
    @@ -17,135 +17,541 @@
      */
     package org.apache.storm.daemon.supervisor;
     
    +import java.io.File;
    +import java.io.IOException;
    +import java.net.UnknownHostException;
    +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.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
     import org.apache.commons.io.FileUtils;
     import org.apache.storm.Config;
     import org.apache.storm.StormTimer;
    -import org.apache.storm.daemon.supervisor.timer.RunProfilerActions;
    +import org.apache.storm.cluster.ClusterStateContext;
    +import org.apache.storm.cluster.ClusterUtils;
    +import org.apache.storm.cluster.DaemonType;
    +import org.apache.storm.cluster.IStateStorage;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.daemon.DaemonCommon;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
     import org.apache.storm.daemon.supervisor.timer.SupervisorHealthCheck;
     import org.apache.storm.daemon.supervisor.timer.SupervisorHeartbeat;
     import org.apache.storm.daemon.supervisor.timer.UpdateBlobs;
    +import org.apache.storm.event.EventManager;
     import org.apache.storm.event.EventManagerImp;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.AsyncLocalizer;
    +import org.apache.storm.localizer.ILocalizer;
     import org.apache.storm.localizer.Localizer;
     import org.apache.storm.messaging.IContext;
     import org.apache.storm.metric.StormMetricsRegistry;
     import org.apache.storm.scheduler.ISupervisor;
     import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
     import org.apache.storm.utils.Utils;
     import org.apache.storm.utils.VersionInfo;
    +import org.apache.zookeeper.data.ACL;
    +import org.eclipse.jetty.util.ConcurrentHashSet;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -import java.io.File;
    -import java.io.InterruptedIOException;
    -import java.util.Collection;
    -import java.util.Map;
    -import java.util.Set;
    -import java.util.concurrent.Callable;
    +public class Supervisor implements DaemonCommon, AutoCloseable {
    +    
    +    public class ReadStateThread implements Runnable, AutoCloseable {
    +        private final Map<String, Object> superConf;
    +        private final IStormClusterState stormClusterState;
    +        private final EventManager syncSupEventManager;
    +        private final AtomicReference<Map<String, Map<String, Object>>> assignmentVersions;
    +        private final Map<Integer, Slot> slots = new HashMap<>();
    +        private final AtomicInteger readRetry = new AtomicInteger(0);
    +        private final String assignmentId;
    +        private final ISupervisor iSuper;
    +        private final ILocalizer localizer;
    +        private final ContainerLauncher launcher;
    +        private final String host;
    +        private final LocalState localState;
    +        private final IStormClusterState clusterState;
    +        private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +        
    +        public ReadStateThread(EventManager syncSupEventManager) throws Exception {
    +            this(getConf(), getStormClusterState(), syncSupEventManager,
    +                    getAssignmentId(), getiSupervisor(),
    +                    getAsyncLocalizer(), getHostName(),
    +                    getLocalState(), getStormClusterState(),
    +                    getCurrAssignment(), getSharedContext());
    +        }
    +        
    +        public ReadStateThread(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +                EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +                ILocalizer localizer, String host, LocalState localState,
    +                IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +                IContext sharedContext) throws Exception{
    +            this.superConf = superConf;
    +            this.stormClusterState = stormClusterState;
    +            this.syncSupEventManager = syncSupEventManager;
    +            this.assignmentVersions = new AtomicReference<Map<String, Map<String, Object>>>(new HashMap<String, Map<String, Object>>());
    +            this.assignmentId = assignmentId;
    +            this.iSuper = iSuper;
    +            this.localizer = localizer;
    +            this.host = host;
    +            this.localState = localState;
    +            this.clusterState = clusterState;
    +            this.cachedAssignments = cachedAssignments;
    +            
    +            this.launcher = ContainerLauncher.mk(superConf, assignmentId, sharedContext);
    +            
    +            @SuppressWarnings("unchecked")
    +            List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +            for (Number port: ports) {
    +                slots.put(port.intValue(), mkSlot(port.intValue()));
    +            }
    +        }
    +
    +        private Slot mkSlot(int port) throws Exception {
    +            Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                    localState, clusterState, iSuper, cachedAssignments);
    +            slot.start();
    +            return slot;
    +        }
    +        
    +        @Override
    +        public synchronized void run() {
    +            try {
    +                Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +                List<String> stormIds = stormClusterState.assignments(syncCallback);
    +                Map<String, Map<String, Object>> assignmentsSnapshot =
    +                        getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +                
    +                Map<Integer, LocalAssignment> allAssignments =
    +                        readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +                if (allAssignments == null) {
    +                    //Something odd happened try again later
    +                    return;
    +                }
    +                Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +                
    +                HashSet<Integer> assignedPorts = new HashSet<>();
    +                LOG.debug("Synchronizing supervisor");
    +                LOG.debug("All assignment: {}", allAssignments);
    +                LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +                for (Integer port: allAssignments.keySet()) {
    +                    if (iSuper.confirmAssigned(port)) {
    +                        assignedPorts.add(port);
    +                    }
    +                }
    +                HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +                allPorts.addAll(slots.keySet());
    +                
    +                Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +                for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                    String topoId = entry.getKey();
    +                    if (entry.getValue() != null) {
    +                        for (ProfileRequest req: entry.getValue()) {
    +                            NodeInfo ni = req.get_nodeInfo();
    +                            if (host.equals(ni.get_node())) {
    +                                Long port = ni.get_port().iterator().next();
    +                                Set<TopoProfileAction> actions = filtered.get(port);
    +                                if (actions == null) {
    +                                    actions = new HashSet<>();
    +                                    filtered.put(port.intValue(), actions);
    +                                }
    +                                actions.add(new TopoProfileAction(topoId, req));
    +                            }
    +                        }
    +                    }
    +                }
    +                
    +                for (Integer port: allPorts) {
    +                    Slot slot = slots.get(port);
    +                    if (slot == null) {
    +                        slot = mkSlot(port);
    +                        slots.put(port, slot);
    +                    }
    +                    slot.setNewAssignment(allAssignments.get(port));
    +                    slot.addProfilerActions(filtered.get(port));
    +                }
    +                
    +            } catch (Exception e) {
    +                LOG.error("Failed to Sync Supervisor", e);
    +                throw new RuntimeException(e);
    +            }
    +        }
    +        
    +        protected Map<String, Map<String, Object>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +                Map<String, Map<String, Object>> localAssignmentVersion, Runnable callback) throws Exception {
    +            Map<String, Map<String, Object>> updateAssignmentVersion = new HashMap<>();
    +            for (String topoId : topoIds) {
    +                Integer recordedVersion = -1;
    +                Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +                if (localAssignmentVersion.containsKey(topoId) && localAssignmentVersion.get(topoId) != null) {
    +                    recordedVersion = (Integer) localAssignmentVersion.get(topoId).get(IStateStorage.VERSION);
    +                }
    +                if (version == null) {
    +                    // ignore
    +                } else if (version == recordedVersion) {
    +                    updateAssignmentVersion.put(topoId, localAssignmentVersion.get(topoId));
    +                } else {
    +                    //TODO change this so we return an object not a map with magic keys
    +                    Map<String, Object> assignmentVersion = (Map<String, Object>) stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                    updateAssignmentVersion.put(topoId, assignmentVersion);
    +                }
    +            }
    +            return updateAssignmentVersion;
    +        }
    +        
    +        protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +            Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +            for (String stormId : stormIds) {
    +                List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +                ret.put(stormId, profileRequests);
    +            }
    +            return ret;
    +        }
    +        
    +        protected Map<Integer, LocalAssignment> readAssignments(Map<String, Map<String, Object>> assignmentsSnapshot,
    +                String assignmentId, AtomicInteger retries) {
    +            try {
    +                Map<Integer, LocalAssignment> portLA = new HashMap<Integer, LocalAssignment>();
    +                for (Map.Entry<String, Map<String, Object>> assignEntry : assignmentsSnapshot.entrySet()) {
    +                    String topoId = assignEntry.getKey();
    +                    Assignment assignment = (Assignment) assignEntry.getValue().get(IStateStorage.DATA);
    +
    +                    Map<Integer, LocalAssignment> portTasks = readMyExecutors(topoId, assignmentId, assignment);
    +
    +                    for (Map.Entry<Integer, LocalAssignment> entry : portTasks.entrySet()) {
    +
    +                        Integer port = entry.getKey();
    +
    +                        LocalAssignment la = entry.getValue();
    +
    +                        if (!portLA.containsKey(port)) {
    +                            portLA.put(port, la);
    +                        } else {
    +                            throw new RuntimeException("Should not have multiple topologys assigned to one port");
    +                        }
    +                    }
    +                }
    +                retries.set(0);
    +                return portLA;
    +            } catch (RuntimeException e) {
    +                if (retries.get() > 2) {
    --- End diff --
    
    this should be config


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78551282
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,420 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + * TODO once we have replaced the original supervisor merge this with
    + * Localizer and optimize them
    + */
    +public class AsyncLocalizer implements ILocalizer, Shutdownable {
    +    /**
    +     * A future that has already completed.
    +     */
    +    private static class AllDoneFuture implements Future<Void> {
    +
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isCancelled() {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isDone() {
    +            return true;
    +        }
    +
    +        @Override
    +        public Void get() {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void get(long timeout, TimeUnit unit) {
    +            return null;
    +        }
    +
    +    }
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizer.class);
    +
    +    private final Localizer _localizer;
    +    private final ExecutorService _execService;
    +    private final boolean _isLocalMode;
    +    private final Map<String, Object> _conf;
    +    private final Map<String, LocalDownloadedResource> _basicPending;
    +    private final Map<String, LocalDownloadedResource> _blobPending;
    +    private final AdvancedFSOps _fsOps;
    +
    +    private class DownloadBaseBlobsDistributed implements Callable<Void> {
    +        private final String _topologyId;
    +        
    +        public DownloadBaseBlobsDistributed(String topologyId) {
    +            this._topologyId = topologyId;
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            File sr = new File(stormroot);
    +            if (sr.exists()) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    Utils.forceDelete(stormroot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            try {
    +                String stormJarKey = ConfigUtils.masterStormJarKey(_topologyId);
    +                String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +                String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +                String jarPath = ConfigUtils.supervisorStormJarPath(tmproot);
    +                String codePath = ConfigUtils.supervisorStormCodePath(tmproot);
    +                String confPath = ConfigUtils.supervisorStormConfPath(tmproot);
    +                FileUtils.forceMkdir(new File(tmproot));
    +                _fsOps.restrictDirectoryPermissions(tmproot);
    +                ClientBlobStore blobStore = Utils.getClientBlobStoreForSupervisor(_conf);
    +                try {
    +                    Utils.downloadResourcesAsSupervisor(stormJarKey, jarPath, blobStore);
    +                    Utils.downloadResourcesAsSupervisor(stormCodeKey, codePath, blobStore);
    +                    Utils.downloadResourcesAsSupervisor(stormConfKey, confPath, blobStore);
    +                } finally {
    +                    blobStore.shutdown();
    +                }
    +                Utils.extractDirFromJar(jarPath, ConfigUtils.RESOURCES_SUBDIR, tmproot);
    +                _fsOps.moveDirectoryPreferAtomic(new File(tmproot), new File(stormroot));
    +                SupervisorUtils.setupStormCodeDir(_conf, ConfigUtils.readSupervisorStormConf(_conf, _topologyId), stormroot);
    +                deleteAll = false;
    +            } finally {
    +                if (deleteAll) {
    +                    LOG.info("Failed to download basic resources for topology-id {}", _topologyId);
    +                    Utils.forceDelete(tmproot);
    +                    Utils.forceDelete(stormroot);
    +                }
    +            }
    +            return null;
    +        }
    +    }
    +    
    +    private class DownloadBaseBlobsLocal implements Callable<Void> {
    +        private final String _topologyId;
    +        
    +        public DownloadBaseBlobsLocal(String topologyId) {
    +            this._topologyId = topologyId;
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            File sr = new File(stormroot);
    +            if (sr.exists()) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    Utils.forceDelete(stormroot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            try {
    +                BlobStore blobStore = Utils.getNimbusBlobStore(_conf, null, null);
    +                FileOutputStream codeOutStream = null;
    +                FileOutputStream confOutStream = null;
    +                try {
    +                    FileUtils.forceMkdir(new File(tmproot));
    +                    String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +                    String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +                    String codePath = ConfigUtils.supervisorStormCodePath(tmproot);
    +                    String confPath = ConfigUtils.supervisorStormConfPath(tmproot);
    +                    codeOutStream = new FileOutputStream(codePath);
    +                    blobStore.readBlobTo(stormCodeKey, codeOutStream, null);
    +                    confOutStream = new FileOutputStream(confPath);
    +                    blobStore.readBlobTo(stormConfKey, confOutStream, null);
    +                } finally {
    +                    if (codeOutStream != null)
    +                        codeOutStream.close();
    +                    if (confOutStream != null)
    +                        codeOutStream.close();
    +                    blobStore.shutdown();
    +                }
    +
    +                ClassLoader classloader = Thread.currentThread().getContextClassLoader();
    +                String resourcesJar = AsyncLocalizer.resourcesJar();
    +                URL url = classloader.getResource(ConfigUtils.RESOURCES_SUBDIR);
    +
    +                String targetDir = tmproot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +
    +                if (resourcesJar != null) {
    +                    LOG.info("Extracting resources from jar at {} to {}", resourcesJar, targetDir);
    +                    Utils.extractDirFromJar(resourcesJar, ConfigUtils.RESOURCES_SUBDIR, stormroot);
    +                } else if (url != null) {
    +                    LOG.info("Copying resources at {} to {} ", url.toString(), targetDir);
    +                    if (url.getProtocol() == "jar") {
    +                        JarURLConnection urlConnection = (JarURLConnection) url.openConnection();
    +                        Utils.extractDirFromJar(urlConnection.getJarFileURL().getFile(), ConfigUtils.RESOURCES_SUBDIR, stormroot);
    +                    } else {
    +                        FileUtils.copyDirectory(new File(url.getFile()), (new File(targetDir)));
    +                    }
    +                }
    +                _fsOps.moveDirectoryPreferAtomic(new File(tmproot), new File(stormroot));
    +                SupervisorUtils.setupStormCodeDir(_conf, ConfigUtils.readSupervisorStormConf(_conf, _topologyId), stormroot);
    +                deleteAll = false;
    +            } finally {
    +                if (deleteAll) {
    +                    LOG.info("Failed to download basic resources for topology-id {}", _topologyId);
    +                    Utils.forceDelete(tmproot);
    +                    Utils.forceDelete(stormroot);
    +                }
    +            }
    +            return null;
    +        }
    +    }
    +    
    +    private class DownloadBlobs implements Callable<Void> {
    +        private final String _topologyId;
    +
    +        public DownloadBlobs(String topologyId) {
    +            _topologyId = topologyId;
    +        }
    +
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            Map<String, Object> stormConf = ConfigUtils.readSupervisorStormConf(_conf, _topologyId);
    +                
    +            @SuppressWarnings("unchecked")
    +            Map<String, Map<String, Object>> blobstoreMap = (Map<String, Map<String, Object>>) stormConf.get(Config.TOPOLOGY_BLOBSTORE_MAP);
    +            String user = (String) stormConf.get(Config.TOPOLOGY_SUBMITTER_USER);
    +            String topoName = (String) stormConf.get(Config.TOPOLOGY_NAME);
    +
    +            List<LocalResource> localResourceList = new ArrayList<>();
    +            if (blobstoreMap != null) {
    +                List<LocalResource> tmp = SupervisorUtils.blobstoreMapToLocalresources(blobstoreMap);
    +                if (tmp != null) {
    +                    localResourceList.addAll(tmp);
    +                }
    +            }
    +
    +            StormTopology stormCode = ConfigUtils.readSupervisorTopology(_conf, _topologyId, _fsOps);
    +            List<String> dependencies = new ArrayList<>();
    +            if (stormCode.is_set_dependency_jars()) {
    +                dependencies.addAll(stormCode.get_dependency_jars());
    +            }
    +            if (stormCode.is_set_dependency_artifacts()) {
    +                dependencies.addAll(stormCode.get_dependency_artifacts());
    +            }
    +            for (String dependency : dependencies) {
    +                localResourceList.add(new LocalResource(dependency, false));
    +            }
    +            
    +            if (!localResourceList.isEmpty()) {
    +                File userDir = _localizer.getLocalUserFileCacheDir(user);
    +                if (!userDir.exists()) {
    +                    FileUtils.forceMkdir(userDir);
    +                }
    +                List<LocalizedResource> localizedResources = _localizer.getBlobs(localResourceList, user, topoName, userDir);
    +                _fsOps.setupBlobPermissions(userDir, user);
    +                for (LocalizedResource localizedResource : localizedResources) {
    +                    File rsrcFilePath = new File(localizedResource.getFilePath());
    +                    String keyName = rsrcFilePath.getName();
    +                    String blobSymlinkTargetName = new File(localizedResource.getCurrentSymlinkPath()).getName();
    +
    +                    String symlinkName = null;
    +                    Map<String, Object> blobInfo = blobstoreMap.get(keyName);
    --- End diff --
    
    @revans2 It's broken again from latest version of the fix. (regression)


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77418129
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,420 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + * TODO once we have replaced the original supervisor merge this with
    + * Localizer and optimize them
    --- End diff --
    
    Yes, we want to do this. We should track this with a jira instead.


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

[GitHub] storm issue #1642: DO NOT MERGE: Please review STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @harshach @knusbaum I think I addressed all of your review comments along with upmerging to the latest master.  I still have a bit more to do, mostly with writing a replacement for supervisor_test.clj but it is fairly close to what I think is a final solution.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76802455
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,629 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +
    +    //For testing purposes
    +    public BasicContainer(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager, LocalState localState,
    +            String profileCmd) throws IOException {
    +        super(ops, port, assignment, conf, topoConf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +        _profileCmd = profileCmd;
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState localState, ResourceIsolationInterface resourceIsolationManager, boolean recover)
    +            throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException(
    +                    "Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            _localState.setApprovedWorkers(workersToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        setup();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = _ops.slurpString(new File(str)).trim();
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        if (command.size() > 0) {
    +            return runProfilingCommand(command, env, logPrefix, targetFile);
    +        }
    +        LOG.warn("PROFILING REQUEST NOT SUPPORTED {} IGNORED...", request);
    +        return true;
    +    }
    +
    +    /**
    +     * Get the command to run when doing profiling
    +     * @param action the profiling action to perform
    +     * @param stop if this is meant to stop the profiling or start it
    +     * @param workerPid the PID of the process to profile
    +     * @param targetDir the current working directory of the worker process
    +     * @return the command to run for profiling.
    +     */
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        switch(action) {
    +            case JMAP_DUMP:
    +                return jmapDumpCmd(workerPid, targetDir);
    +            case JSTACK_DUMP:
    +                return jstackDumpCmd(workerPid, targetDir);
    +            case JPROFILE_DUMP:
    +                return jprofileDump(workerPid, targetDir);
    +            case JVM_RESTART:
    +                return jprofileJvmRestart(workerPid);
    +            case JPROFILE_STOP:
    +                if (stop) {
    +                    return jprofileStop(workerPid, targetDir);
    +                }
    +                return jprofileStart(workerPid);
    +            default:
    +                return Lists.newArrayList();
    +        }
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +
    +    /**
    +     * Compute the java.library.path that should be used for the worker.
    +     * This helps it to load JNI libraries that are packaged in the uber jar.
    +     * @param stormRoot the root directory of the worker process
    +     * @param conf the config for the supervisor.
    +     * @return the java.library.path/LD_LIBRARY_PATH to use so native libraries load correctly.
    +     */
    +    protected String jlp(String stormRoot, Map<String, Object> conf) {
    +        String resourceRoot = stormRoot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +        String os = System.getProperty("os.name").replaceAll("\\s+", "_");
    +        String arch = System.getProperty("os.arch");
    +        String archResourceRoot = resourceRoot + Utils.FILE_PATH_SEPARATOR + os + "-" + arch;
    +        String ret = CPJ.join(archResourceRoot, resourceRoot,
    +                conf.get(Config.JAVA_LIBRARY_PATH));
    +        return ret;
    +    }
    +
    +    /**
    +     * Returns a collection of jar file names found under the given directory.
    +     * @param dir the directory to search
    +     * @return the jar file names
    +     */
    +    protected List<String> getFullJars(File dir) {
    +        File[] files = dir.listFiles(jarFilter);
    +
    +        if (files == null) {
    +            return Collections.emptyList();
    +        }
    +
    +        return Arrays.stream(files).map(f -> f.getAbsolutePath())
    +                .collect(Collectors.toList());
    +    }
    +    
    +    protected List<String> frameworkClasspath() {
    +        String stormHome = System.getProperty("storm.home");
    +
    +        File stormLibDir = new File(stormHome, "lib");
    +        String stormConfDir =
    +                System.getenv("STORM_CONF_DIR") != null ?
    +                System.getenv("STORM_CONF_DIR") :
    +                new File(stormHome, "conf").getAbsolutePath();
    +        File stormExtlibDir = new File(stormHome, "extlib");
    +        String extcp = System.getenv("STORM_EXT_CLASSPATH");
    +        List<String> pathElements = new LinkedList<>();
    +        pathElements.addAll(getFullJars(stormLibDir));
    +        pathElements.addAll(getFullJars(stormExtlibDir));
    +        pathElements.add(extcp);
    +        pathElements.add(stormConfDir);
    +
    +        return pathElements;
    +    }
    +    
    +    @SuppressWarnings("unchecked")
    +    private List<String> asStringList(Object o) {
    +        if (o instanceof String) {
    +            return Arrays.asList((String)o);
    +        } else if (o instanceof List) {
    +            return (List<String>)o;
    +        }
    +        return Collections.EMPTY_LIST;
    +    }
    +    
    +    /**
    +     * Compute the classpath for the worker process
    +     * @param stormJar the topology jar
    +     * @param dependencyLocations any dependencies from the topology
    +     * @return the full classpath
    +     */
    +    protected String getWorkerClassPath(String stormJar, List<String> dependencyLocations) {
    +        List<String> workercp = new ArrayList<>();
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH_BEGINNING)));
    +        workercp.addAll(frameworkClasspath());
    +        workercp.add(stormJar);
    +        workercp.addAll(dependencyLocations);
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH)));
    +        return CPJ.join(workercp);
    +    }
    +
    +    private String substituteChildOptsInternal(String string, int memOnheap) {
    +        if (StringUtils.isNotBlank(string)) {
    +            String p = String.valueOf(_port);
    +            string = string.replace("%ID%", p);
    +            string = string.replace("%WORKER-ID%", _workerId);
    +            string = string.replace("%TOPOLOGY-ID%", _topologyId);
    +            string = string.replace("%WORKER-PORT%", p);
    +            if (memOnheap > 0) {
    +                string = string.replace("%HEAP-MEM%", String.valueOf(memOnheap));
    +            }
    +        }
    +        return string;
    +    }
    +    
    +    protected List<String> substituteChildopts(Object value) {
    +        return substituteChildopts(value, -1);
    +    }
    +
    +    protected List<String> substituteChildopts(Object value, int memOnheap) {
    +        List<String> rets = new ArrayList<>();
    +        if (value instanceof String) {
    +            String string = substituteChildOptsInternal((String) value, memOnheap);
    +            if (StringUtils.isNotBlank(string)) {
    +                String[] strings = string.split("\\s+");
    +                rets.addAll(Arrays.asList(strings));
    +            }
    +        } else if (value instanceof List) {
    +            @SuppressWarnings("unchecked")
    +            List<String> objects = (List<String>) value;
    +            for (String object : objects) {
    +                String str = substituteChildOptsInternal(object, memOnheap);
    +                if (StringUtils.isNotBlank(str)) {
    +                    rets.add(str);
    +                }
    +            }
    +        }
    +        return rets;
    +    }
    +
    +    /**
    +     * Launch the worker process (non-blocking)
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param processExitcallback
    +     *            a callback for when the process exits
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     */
    +    protected void launchWorkerProcess(List<String> command, Map<String, String> env, String logPrefix,
    +            ExitCodeCallback processExitCallback, File targetDir) throws IOException {
    +        SupervisorUtils.launchProcess(command, env, logPrefix, processExitCallback, targetDir);
    +    }
    +
    +    private String getWorkerLoggingConfigFile(String stormHome) {
    +        String log4jConfigurationDir = (String) (_conf.get(Config.STORM_LOG4J2_CONF_DIR));
    +
    +        if (StringUtils.isNotBlank(log4jConfigurationDir)) {
    +            if (!Utils.isAbsolutePath(log4jConfigurationDir)) {
    +                log4jConfigurationDir = stormHome + Utils.FILE_PATH_SEPARATOR + log4jConfigurationDir;
    +            }
    +        } else {
    +            log4jConfigurationDir = stormHome + Utils.FILE_PATH_SEPARATOR + "log4j2";
    +        }
    +        
    +
    +        if (Utils.IS_ON_WINDOWS && !log4jConfigurationDir.startsWith("file:")) {
    +            log4jConfigurationDir = "file:///" + log4jConfigurationDir;
    +        }
    +        return log4jConfigurationDir + Utils.FILE_PATH_SEPARATOR + "worker.xml";
    +    }
    +    
    +    /**
    +     * Get parameters for the class path of the worker process.  Also used by the
    +     * log Writer
    +     * @param stormRoot the root dist dir for the topology
    +     * @return the classpath for the topology as command line arguments.
    +     * @throws IOException on any error.
    +     */
    +    private List<String> getClassPathParams(final String stormRoot) throws IOException {
    +        final String stormJar = ConfigUtils.supervisorStormJarPath(stormRoot);
    +        final StormTopology stormTopology = ConfigUtils.readSupervisorTopology(_conf, _topologyId, _ops);
    +        final List<String> dependencyLocations = new ArrayList<>();
    +        if (stormTopology.get_dependency_jars() != null) {
    +            for (String dependency : stormTopology.get_dependency_jars()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +
    +        if (stormTopology.get_dependency_artifacts() != null) {
    +            for (String dependency : stormTopology.get_dependency_artifacts()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +        final String workerClassPath = getWorkerClassPath(stormJar, dependencyLocations);
    +        
    +        List<String> classPathParams = new ArrayList<>();
    +        classPathParams.add("-cp");
    +        classPathParams.add(workerClassPath);
    +        return classPathParams;
    +    }
    +    
    +    /**
    +     * Get a set of java properties that are common to both the log writer and the worker processes.
    +     * These are mostly system properties that are used by logging.
    +     * @return a list of command line options
    +     */
    +    private List<String> getCommonParams() {
    +        final String stormHome = ConfigUtils.concatIfNotNull(System.getProperty("storm.home"));
    +        final String workersArtifacts = ConfigUtils.workerArtifactsRoot(_conf);
    +        String stormLogDir = ConfigUtils.getLogDir();
    +        String log4jConfigurationFile = getWorkerLoggingConfigFile(stormHome);
    +        
    +        List<String> commonParams = new ArrayList<>();
    +        commonParams.add("-Dlogfile.name=worker.log");
    +        commonParams.add("-Dstorm.home=" + stormHome);
    +        commonParams.add("-Dworkers.artifacts=" + workersArtifacts);
    +        commonParams.add("-Dstorm.id=" + _topologyId);
    +        commonParams.add("-Dworker.id=" + _workerId);
    +        commonParams.add("-Dworker.port=" + _port);
    +        commonParams.add("-Dstorm.log.dir=" + stormLogDir);
    +        commonParams.add("-Dlog4j.configurationFile=" + log4jConfigurationFile);
    +        commonParams.add("-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector");
    +        return commonParams;
    +    }
    +    
    +    private int getMemOnHeap(WorkerResources resources) {
    +        int memOnheap = 0;
    +        if (resources != null && resources.is_set_mem_on_heap() && 
    +                resources.get_mem_on_heap() > 0) {
    +            memOnheap = (int) Math.ceil(resources.get_mem_on_heap());
    +        } else {
    +            // set the default heap memory size for supervisor-test
    +            memOnheap = Utils.getInt(_topoConf.get(Config.WORKER_HEAP_MEMORY_MB), 768);
    +        }
    +        return memOnheap;
    +    }
    +    
    +    private List<String> getWorkerProfilerChildOpts(int memOnheap) {
    +        List<String> workerProfilerChildopts = new ArrayList<>();
    +        if (Utils.getBoolean(_conf.get(Config.WORKER_PROFILER_ENABLED), false)) {
    +            workerProfilerChildopts = substituteChildopts(_conf.get(Config.WORKER_PROFILER_CHILDOPTS), memOnheap);
    +        }
    +        return workerProfilerChildopts;
    +    }
    +    
    +    /**
    +     * a or b the first one that is not null
    +     * @param a something
    +     * @param b something else
    +     * @return a or b the first one that is not null
    +     */
    +    private <V> V OR(V a, V b) {
    +        return a == null ? b : a;
    +    }
    +    
    +    protected String javaCmd(String cmd) {
    +        String ret = null;
    +        String javaHome = System.getenv().get("JAVA_HOME");
    +        if (StringUtils.isNotBlank(javaHome)) {
    +            ret = javaHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR + cmd;
    +        } else {
    +            ret = cmd;
    +        }
    +        return ret;
    +    }
    +    
    +    /**
    +     * Create the command to launch the worker process
    +     * @param memOnheap the on heap memory for the worker
    +     * @param stormRoot the root dist dir for the topology
    +     * @param jlp java library path for the topology
    +     * @return the command to run
    +     * @throws IOException on any error.
    +     */
    +    private List<String> mkLaunchCommand(final int memOnheap, final String stormRoot,
    +            final String jlp) throws IOException {
    +        final String javaCmd = javaCmd("java");
    +        final String stormOptions = ConfigUtils.concatIfNotNull(System.getProperty("storm.options"));
    +        final String stormConfFile = ConfigUtils.concatIfNotNull(System.getProperty("storm.conf.file"));
    +        final String workerTmpDir = ConfigUtils.workerTmpRoot(_conf, _workerId);
    +        
    +        List<String> classPathParams = getClassPathParams(stormRoot);
    +        List<String> commonParams = getCommonParams();
    +        
    +        List<String> commandList = new ArrayList<>();
    +        //Log Writer Command...
    +        commandList.add(javaCmd);
    +        commandList.addAll(classPathParams);
    +        commandList.addAll(substituteChildopts(_topoConf.get(Config.TOPOLOGY_WORKER_LOGWRITER_CHILDOPTS)));
    +        commandList.addAll(commonParams);
    +        commandList.add("org.apache.storm.LogWriter"); //The LogWriter in turn launches the actual worker.
    +
    +        //Worker Command...
    +        commandList.add(javaCmd);
    +        commandList.add("-server");
    +        commandList.addAll(commonParams);
    +        commandList.addAll(substituteChildopts(_conf.get(Config.WORKER_CHILDOPTS), memOnheap));
    +        commandList.addAll(substituteChildopts(_topoConf.get(Config.TOPOLOGY_WORKER_CHILDOPTS), memOnheap));
    +        commandList.addAll(substituteChildopts(OR(
    +                _topoConf.get(Config.TOPOLOGY_WORKER_GC_CHILDOPTS),
    +                _conf.get(Config.WORKER_GC_CHILDOPTS)), memOnheap));
    +        commandList.addAll(getWorkerProfilerChildOpts(memOnheap));
    +        commandList.add("-Djava.library.path=" + jlp);
    +        commandList.add("-Dstorm.conf.file=" + stormConfFile);
    +        commandList.add("-Dstorm.options=" + stormOptions);
    +        commandList.add("-Djava.io.tmpdir=" + workerTmpDir);
    +        commandList.add("-Dlogging.sensitivity=" + OR((String) _topoConf.get(Config.TOPOLOGY_LOGGING_SENSITIVITY), "S3"));
    +        commandList.addAll(classPathParams);
    +        commandList.add("org.apache.storm.daemon.worker");
    +        commandList.add(_topologyId);
    +        commandList.add(_supervisorId);
    +        commandList.add(String.valueOf(_port));
    +        commandList.add(_workerId);
    +        
    +        return commandList;
    +    }
    +    
    +    @Override
    +    public void launch() throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot launch a container recovered with just a worker id");
    +        }
    +        LOG.info("Launching worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                _supervisorId, _port, _workerId);
    +        String logPrefix = "Worker Process " + _workerId;
    +        ProcessExitCallback processExitCallback = new ProcessExitCallback(logPrefix);
    +        _exitedEarly = false;
    +        
    +        final WorkerResources resources = _assignment.get_resources();
    +        final int memOnheap = getMemOnHeap(resources);
    +        final String stormRoot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +        final String jlp = jlp(stormRoot, _conf);
    +        
    +        List<String> commandList = mkLaunchCommand(memOnheap, stormRoot, jlp);
    +
    +        Map<String, String> topEnvironment = new HashMap<String, String>();
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> environment = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (environment != null) {
    +            topEnvironment.putAll(environment);
    +        }
    +        topEnvironment.put("LD_LIBRARY_PATH", jlp);
    +
    +        // {"cpu" cpu "memory" (+ mem-onheap mem-offheap (int (Math/ceil (conf
    --- End diff --
    
    remove?


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    Looks like there were two failures in travis.  One is a rat issue with storm-submit-tools the other is an integration test that timed out.  I'll try to reproduce the issues and see if I can fix them.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77415816
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    --- End diff --
    
    I tried that, and for some reason java doesn't like it, because I am initializing it with a HashMap, that is a Map, but not exactly a Map. I wasn't totally sure why java didn't like it.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77413129
  
    --- 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);
    +            }
    +            if (container != null) {
    +                sb.append(" container: ");
    +                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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WAITING_FOR_WORKER_START) {
    +                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;
    +                    } else 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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), 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.get_topology_id(), 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);
    +        
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        //The child processes typically exit in < 1 sec.  If 2 mins later they are still around something is wrong
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for WAITING_FOR_WORKER_START state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleWaitingForWorkerStart(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb != null) {
    +            long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +            if (hbAgeMs <= staticState.hbTimeoutMs) {
    +                return dynamicState.withState(MachineState.RUNNING);
    +            }
    +        }
    +        
    +        if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            //We were rescheduled while waiting for the worker to come up
    +            return Slot.killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffms = (Time.currentTimeMillis() - dynamicState.startTime);
    +        if (timeDiffms > staticState.firstHbTimeoutMs) {
    +            LOG.warn("SLOT {}: Container {} failed to launch in {} ms.", staticState.port, dynamicState.container, staticState.firstHbTimeoutMs);
    +            dynamicState.container.kill();
    +            Time.sleep(staticState.killSleepMs);
    +            return dynamicState.withState(MachineState.KILL_AND_RELAUNCH);
    --- End diff --
    
    This satisfies the precondition for `handleKillAndRelaunch`.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79035641
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,328 @@
    +/**
    + * 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.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 java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +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.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this.superConf = supervisor.getConf();
    +        this.stormClusterState = supervisor.getStormClusterState();
    +        this.syncSupEventManager = supervisor.getEventManger();
    +        this.assignmentVersions = new AtomicReference<>(new HashMap<>());
    +        this.assignmentId = supervisor.getAssignmentId();
    +        this.iSuper = supervisor.getiSupervisor();
    +        this.localizer = supervisor.getAsyncLocalizer();
    +        this.host = supervisor.getHostName();
    +        this.localState = supervisor.getLocalState();
    +        this.clusterState = supervisor.getStormClusterState();
    +        this.cachedAssignments = supervisor.getCurrAssignment();
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, supervisor.getSharedContext());
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +        
    +        try {
    +            Collection<String> workers = SupervisorUtils.supervisorWorkerIds(superConf);
    +            for (Slot slot: slots.values()) {
    +                String workerId = slot.getWorkerId();
    +                if (workerId != null) {
    +                    workers.remove(workerId);
    +                }
    +            }
    +            if (!workers.isEmpty()) {
    +                supervisor.killWorkers(workers, launcher);
    +            }
    +        } catch (Exception e) {
    +            LOG.warn("Error trying to clean up old workers", e);
    +        }
    +
    +        //All the slots/assignments should be recovered now, so we can clean up anything that we don't expect to be here
    +        try {
    +            localizer.cleanupUnusedTopologies();
    +        } catch (Exception e) {
    +            LOG.warn("Error trying to clean up old topologies", e);
    +        }
    +        
    +        for (Slot slot: slots.values()) {
    +            slot.start();
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        return new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    +                            if (actions == null) {
    +                                actions = new HashSet<>();
    +                                filtered.put(port.intValue(), actions);
    +                            }
    +                            actions.add(new TopoProfileAction(topoId, req));
    +                        }
    +                    }
    +                }
    +            }
    +            
    +            for (Integer port: allPorts) {
    +                Slot slot = slots.get(port);
    +                if (slot == null) {
    +                    slot = mkSlot(port);
    +                    slots.put(port, slot);
    +                    slot.start();
    +                }
    +                slot.setNewAssignment(allAssignments.get(port));
    +                slot.addProfilerActions(filtered.get(port));
    +            }
    +            
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +    }
    +    
    +    protected Map<String, VersionedData<Assignment>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +            Map<String, VersionedData<Assignment>> localAssignmentVersion, Runnable callback) throws Exception {
    +        Map<String, VersionedData<Assignment>> updateAssignmentVersion = new HashMap<>();
    +        for (String topoId : topoIds) {
    +            Integer recordedVersion = -1;
    +            Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +            VersionedData<Assignment> locAssignment = localAssignmentVersion.get(topoId);
    +            if (locAssignment != null) {
    +                recordedVersion = locAssignment.getVersion();
    +            }
    +            if (version == null) {
    +                // ignore
    +            } else if (version == recordedVersion) {
    +                updateAssignmentVersion.put(topoId, locAssignment);
    +            } else {
    +                VersionedData<Assignment> assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                updateAssignmentVersion.put(topoId, assignmentVersion);
    +            }
    +        }
    +        return updateAssignmentVersion;
    +    }
    +    
    +    protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +        Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +        for (String stormId : stormIds) {
    +            List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +            ret.put(stormId, profileRequests);
    +        }
    +        return ret;
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readAssignments(Map<String, VersionedData<Assignment>> assignmentsSnapshot,
    +            String assignmentId, AtomicInteger retries) {
    --- End diff --
    
    It wasn't so much that it uses a retries counter, I was just wondering why the method doesn't refer to the readRetry field directly. It's probably fine 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77738006
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Supervisor.java ---
    @@ -17,135 +17,231 @@
      */
     package org.apache.storm.daemon.supervisor;
     
    +import java.io.File;
    +import java.io.IOException;
    +import java.net.UnknownHostException;
    +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 java.util.concurrent.Callable;
    +import java.util.concurrent.atomic.AtomicReference;
    +
     import org.apache.commons.io.FileUtils;
     import org.apache.storm.Config;
     import org.apache.storm.StormTimer;
    -import org.apache.storm.daemon.supervisor.timer.RunProfilerActions;
    +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.supervisor.timer.SupervisorHealthCheck;
     import org.apache.storm.daemon.supervisor.timer.SupervisorHeartbeat;
     import org.apache.storm.daemon.supervisor.timer.UpdateBlobs;
    +import org.apache.storm.event.EventManager;
     import org.apache.storm.event.EventManagerImp;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.localizer.AsyncLocalizer;
    +import org.apache.storm.localizer.ILocalizer;
     import org.apache.storm.localizer.Localizer;
     import org.apache.storm.messaging.IContext;
     import org.apache.storm.metric.StormMetricsRegistry;
     import org.apache.storm.scheduler.ISupervisor;
     import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
     import org.apache.storm.utils.Utils;
     import org.apache.storm.utils.VersionInfo;
    +import org.apache.zookeeper.data.ACL;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -import java.io.File;
    -import java.io.InterruptedIOException;
    -import java.util.Collection;
    -import java.util.Map;
    -import java.util.Set;
    -import java.util.concurrent.Callable;
    -
    -public class Supervisor {
    +public class Supervisor implements DaemonCommon, AutoCloseable {
         private static final Logger LOG = LoggerFactory.getLogger(Supervisor.class);
    +    private final Map<String, Object> conf;
    +    private final IContext sharedContext;
    +    private volatile boolean active;
    +    private final ISupervisor iSupervisor;
    +    private final Utils.UptimeComputer upTime;
    +    private final String stormVersion;
    +    private final IStormClusterState stormClusterState;
    +    private final LocalState localState;
    +    private final String supervisorId;
    +    private final String assignmentId;
    +    private final String hostName;
    +    // used for reporting used ports when heartbeating
    +    private final AtomicReference<Map<Long, LocalAssignment>> currAssignment;
    --- End diff --
    
    Just fixed it


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77417627
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/RunAsUserContainerLauncher.java ---
    @@ -0,0 +1,71 @@
    +/**
    + * 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.Map;
    +
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.utils.LocalState;
    +
    +public class RunAsUserContainerLauncher extends ContainerLauncher {
    +    private final Map<String, Object> _conf;
    +    private final String _supervisorId;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    public RunAsUserContainerLauncher(Map<String, Object> conf, String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +    }
    +
    +    @Override
    +    public Container launchContainer(int port, LocalAssignment assignment, LocalState state) throws IOException {
    +        Container container = new RunAsUserContainer(port, assignment, _conf, _supervisorId, state,
    +                _resourceIsolationManager, false);
    +        container.setup();
    +        container.launch();
    +        return container;
    +    }
    +
    +    @Override
    +    public Container recoverContainer(int port, LocalAssignment assignment, LocalState state) throws IOException {
    +        Container container = null;
    +        try {
    +            container = new RunAsUserContainer(port, assignment, _conf, _supervisorId, state, 
    +                    _resourceIsolationManager, true);
    +        } catch (ContainerRecoveryException e) {
    +            // We could not recover return null
    +        }
    +        return container;
    +    }
    +    
    +    @Override
    +    public Killable recoverContainer(String workerId) throws IOException {
    +        Container container = null;
    +        try {
    +            container = new RunAsUserContainer(workerId, _conf, _supervisorId, 
    +                    _resourceIsolationManager);
    +        } catch (ContainerRecoveryException e) {
    +            // We could not recover return null
    +        }
    +        return container;
    +    }
    --- End diff --
    
    But then the ContainerRecoveryException will not be turned into a null.  We can change that everywhere, but for now it is an implementation detail that is hidden.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77418240
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,420 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + * TODO once we have replaced the original supervisor merge this with
    + * Localizer and optimize them
    + */
    +public class AsyncLocalizer implements ILocalizer, Shutdownable {
    +    /**
    +     * A future that has already completed.
    +     */
    +    private static class AllDoneFuture implements Future<Void> {
    +
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isCancelled() {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isDone() {
    +            return true;
    +        }
    +
    +        @Override
    +        public Void get() {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void get(long timeout, TimeUnit unit) {
    --- End diff --
    
    My IDE is warning me that the TimeUnit param needs a `@NotNull` annotation or else the requirement gets overridden here, which is probably not intended.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76308561
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,494 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +    
    +    private class ProcessExitCallback implements Utils.ExitCodeCallable<Void> {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public Void call() throws Exception {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +            return null;
    +        }
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, LocalState localState, 
    +            ResourceIsolationInterface resourceIsolationManager, boolean recover) throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized(localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry: workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port +" "+ assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +        
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +    
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized(_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +        synchronized(_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            _localState.setApprovedWorkers(workersToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +    
    +    /**
    +     * Run the given command for profiling
    +     * @param command the command to run
    +     * @param env the environment to run the command
    +     * @param logPrefix the prefix to include in the logs
    +     * @param targetDir the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException on any error
    +     * @throws InterruptedException if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix, File targetDir) throws IOException, InterruptedException {
    +        Process p = Utils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +    
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String topologyId = _assignment.get_topology_id();
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, topologyId, _port);
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) topologyConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, topologyId, _port);
    +
    +        String workerPid = null;
    +        try (FileReader reader = new FileReader(str); BufferedReader br = new BufferedReader(reader)) {
    +            workerPid = br.readLine().trim();
    +        }
    +        
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        return runProfilingCommand(command, env, logPrefix, targetFile);
    +    }
    +    
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        if (action == ProfileAction.JMAP_DUMP) {
    +            return jmapDumpCmd(workerPid, targetDir);
    +        } else if (action == ProfileAction.JSTACK_DUMP) {
    +            return jstackDumpCmd(workerPid, targetDir);
    +        } else if (action == ProfileAction.JPROFILE_DUMP) {
    +            return jprofileDump(workerPid, targetDir);
    +        } else if (action == ProfileAction.JVM_RESTART) {
    +            return jprofileJvmRestart(workerPid);
    +        } else if (!stop && action == ProfileAction.JPROFILE_STOP) {
    +            return jprofileStart(workerPid);
    +        } else if (stop && action == ProfileAction.JPROFILE_STOP) {
    +            return jprofileStop(workerPid, targetDir);
    +        }
    +        return Lists.newArrayList();
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +    
    +    protected String jlp(String stormRoot, Map<String, Object> conf) {
    +        String resourceRoot = stormRoot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +        String os = System.getProperty("os.name").replaceAll("\\s+", "_");
    +        String arch = System.getProperty("os.arch");
    +        String archResourceRoot = resourceRoot + Utils.FILE_PATH_SEPARATOR + os + "-" + arch;
    +        String ret = archResourceRoot + Utils.CLASS_PATH_SEPARATOR + resourceRoot + Utils.CLASS_PATH_SEPARATOR + conf.get(Config.JAVA_LIBRARY_PATH);
    +        return ret;
    +    }
    +    
    +    @SuppressWarnings("unchecked")
    +    protected String getWorkerClassPath(String stormJar, Map<String, Object> topologyConf) {
    +        List<String> topoClasspath = new ArrayList<>();
    +        Object object = topologyConf.get(Config.TOPOLOGY_CLASSPATH);
    +
    +        // Will be populated only if STORM_TOPOLOGY_CLASSPATH_BEGINNING_ENABLED is set on Nimbus.
    +        // Allowed for extreme debugging.
    +        Object topologyClasspathFirst = topologyConf.get(Config.TOPOLOGY_CLASSPATH_BEGINNING);
    +        List<String> firstClasspathList = new ArrayList<>();
    +        if(topologyClasspathFirst instanceof List) {
    +            firstClasspathList.addAll((List<String>)topologyClasspathFirst);
    +        } else if (topologyClasspathFirst instanceof String) {
    +            firstClasspathList.add((String) topologyClasspathFirst);
    +        }
    +        LOG.debug("Topology Classpath Prefix: {}", firstClasspathList);
    +
    +        if (object instanceof List) {
    +            topoClasspath.addAll((List<String>) object);
    +        } else if (object instanceof String) {
    +            topoClasspath.add((String) object);
    +        }
    +        LOG.debug("Topology specific classpath is {}", object);
    +
    +        String classPath = Utils.addToClasspath(firstClasspathList, Arrays.asList(Utils.workerClasspath()));
    +        String classAddPath = Utils.addToClasspath(classPath, Arrays.asList(stormJar));
    +        return Utils.addToClasspath(classAddPath, topoClasspath);
    +    }
    +    
    +    private static String substituteChildOptsInternal(String string,  String workerId, String stormId, int port, int memOnheap) {
    +        if (StringUtils.isNotBlank(string)){
    +            string = string.replace("%ID%", String.valueOf(port));
    +            string = string.replace("%WORKER-ID%", workerId);
    +            string = string.replace("%TOPOLOGY-ID%", stormId);
    +            string = string.replace("%WORKER-PORT%", String.valueOf(port));
    +            string = string.replace("%HEAP-MEM%", String.valueOf(memOnheap));
    +        }
    +        return string;
    +    }
    +    
    +    protected List<String> substituteChildopts(Object value, String workerId, String stormId, int port, int memOnheap) {
    +        List<String> rets = new ArrayList<>();
    +        if (value instanceof String) {
    +            String string = substituteChildOptsInternal((String) value,  workerId, stormId, port, memOnheap);
    +            if (StringUtils.isNotBlank(string)){
    +                String[] strings = string.split("\\s+");
    +                rets.addAll(Arrays.asList(strings));
    +            }
    +        } else if (value instanceof List) {
    +            @SuppressWarnings("unchecked")
    --- End diff --
    
    I put it here because it limits the scope.  That way if someone did something bad in another part of the same method it would still be a warning.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76873268
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,629 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +
    +    //For testing purposes
    +    public BasicContainer(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager, LocalState localState,
    +            String profileCmd) throws IOException {
    +        super(ops, port, assignment, conf, topoConf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +        _profileCmd = profileCmd;
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState localState, ResourceIsolationInterface resourceIsolationManager, boolean recover)
    +            throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException(
    +                    "Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            _localState.setApprovedWorkers(workersToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        setup();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = _ops.slurpString(new File(str)).trim();
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        if (command.size() > 0) {
    +            return runProfilingCommand(command, env, logPrefix, targetFile);
    +        }
    +        LOG.warn("PROFILING REQUEST NOT SUPPORTED {} IGNORED...", request);
    +        return true;
    +    }
    +
    +    /**
    +     * Get the command to run when doing profiling
    +     * @param action the profiling action to perform
    +     * @param stop if this is meant to stop the profiling or start it
    +     * @param workerPid the PID of the process to profile
    +     * @param targetDir the current working directory of the worker process
    +     * @return the command to run for profiling.
    +     */
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        switch(action) {
    +            case JMAP_DUMP:
    +                return jmapDumpCmd(workerPid, targetDir);
    +            case JSTACK_DUMP:
    +                return jstackDumpCmd(workerPid, targetDir);
    +            case JPROFILE_DUMP:
    +                return jprofileDump(workerPid, targetDir);
    +            case JVM_RESTART:
    +                return jprofileJvmRestart(workerPid);
    +            case JPROFILE_STOP:
    +                if (stop) {
    +                    return jprofileStop(workerPid, targetDir);
    +                }
    +                return jprofileStart(workerPid);
    +            default:
    +                return Lists.newArrayList();
    +        }
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +
    +    /**
    +     * Compute the java.library.path that should be used for the worker.
    +     * This helps it to load JNI libraries that are packaged in the uber jar.
    +     * @param stormRoot the root directory of the worker process
    +     * @param conf the config for the supervisor.
    +     * @return the java.library.path/LD_LIBRARY_PATH to use so native libraries load correctly.
    +     */
    +    protected String jlp(String stormRoot, Map<String, Object> conf) {
    +        String resourceRoot = stormRoot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +        String os = System.getProperty("os.name").replaceAll("\\s+", "_");
    +        String arch = System.getProperty("os.arch");
    +        String archResourceRoot = resourceRoot + Utils.FILE_PATH_SEPARATOR + os + "-" + arch;
    +        String ret = CPJ.join(archResourceRoot, resourceRoot,
    +                conf.get(Config.JAVA_LIBRARY_PATH));
    +        return ret;
    +    }
    +
    +    /**
    +     * Returns a collection of jar file names found under the given directory.
    +     * @param dir the directory to search
    +     * @return the jar file names
    +     */
    +    protected List<String> getFullJars(File dir) {
    +        File[] files = dir.listFiles(jarFilter);
    +
    +        if (files == null) {
    +            return Collections.emptyList();
    +        }
    +
    +        return Arrays.stream(files).map(f -> f.getAbsolutePath())
    +                .collect(Collectors.toList());
    +    }
    +    
    +    protected List<String> frameworkClasspath() {
    +        String stormHome = System.getProperty("storm.home");
    +
    +        File stormLibDir = new File(stormHome, "lib");
    +        String stormConfDir =
    +                System.getenv("STORM_CONF_DIR") != null ?
    +                System.getenv("STORM_CONF_DIR") :
    +                new File(stormHome, "conf").getAbsolutePath();
    +        File stormExtlibDir = new File(stormHome, "extlib");
    +        String extcp = System.getenv("STORM_EXT_CLASSPATH");
    +        List<String> pathElements = new LinkedList<>();
    +        pathElements.addAll(getFullJars(stormLibDir));
    +        pathElements.addAll(getFullJars(stormExtlibDir));
    +        pathElements.add(extcp);
    +        pathElements.add(stormConfDir);
    +
    +        return pathElements;
    +    }
    +    
    +    @SuppressWarnings("unchecked")
    +    private List<String> asStringList(Object o) {
    +        if (o instanceof String) {
    +            return Arrays.asList((String)o);
    +        } else if (o instanceof List) {
    +            return (List<String>)o;
    +        }
    +        return Collections.EMPTY_LIST;
    +    }
    +    
    +    /**
    +     * Compute the classpath for the worker process
    +     * @param stormJar the topology jar
    +     * @param dependencyLocations any dependencies from the topology
    +     * @return the full classpath
    +     */
    +    protected String getWorkerClassPath(String stormJar, List<String> dependencyLocations) {
    +        List<String> workercp = new ArrayList<>();
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH_BEGINNING)));
    +        workercp.addAll(frameworkClasspath());
    +        workercp.add(stormJar);
    +        workercp.addAll(dependencyLocations);
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH)));
    +        return CPJ.join(workercp);
    +    }
    +
    +    private String substituteChildOptsInternal(String string, int memOnheap) {
    +        if (StringUtils.isNotBlank(string)) {
    +            String p = String.valueOf(_port);
    +            string = string.replace("%ID%", p);
    +            string = string.replace("%WORKER-ID%", _workerId);
    +            string = string.replace("%TOPOLOGY-ID%", _topologyId);
    +            string = string.replace("%WORKER-PORT%", p);
    +            if (memOnheap > 0) {
    +                string = string.replace("%HEAP-MEM%", String.valueOf(memOnheap));
    +            }
    +        }
    +        return string;
    +    }
    +    
    +    protected List<String> substituteChildopts(Object value) {
    +        return substituteChildopts(value, -1);
    +    }
    +
    +    protected List<String> substituteChildopts(Object value, int memOnheap) {
    +        List<String> rets = new ArrayList<>();
    +        if (value instanceof String) {
    +            String string = substituteChildOptsInternal((String) value, memOnheap);
    +            if (StringUtils.isNotBlank(string)) {
    +                String[] strings = string.split("\\s+");
    +                rets.addAll(Arrays.asList(strings));
    +            }
    +        } else if (value instanceof List) {
    +            @SuppressWarnings("unchecked")
    +            List<String> objects = (List<String>) value;
    +            for (String object : objects) {
    +                String str = substituteChildOptsInternal(object, memOnheap);
    +                if (StringUtils.isNotBlank(str)) {
    +                    rets.add(str);
    +                }
    +            }
    +        }
    +        return rets;
    +    }
    +
    +    /**
    +     * Launch the worker process (non-blocking)
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param processExitcallback
    +     *            a callback for when the process exits
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     */
    +    protected void launchWorkerProcess(List<String> command, Map<String, String> env, String logPrefix,
    +            ExitCodeCallback processExitCallback, File targetDir) throws IOException {
    +        SupervisorUtils.launchProcess(command, env, logPrefix, processExitCallback, targetDir);
    +    }
    +
    +    private String getWorkerLoggingConfigFile(String stormHome) {
    +        String log4jConfigurationDir = (String) (_conf.get(Config.STORM_LOG4J2_CONF_DIR));
    +
    +        if (StringUtils.isNotBlank(log4jConfigurationDir)) {
    +            if (!Utils.isAbsolutePath(log4jConfigurationDir)) {
    +                log4jConfigurationDir = stormHome + Utils.FILE_PATH_SEPARATOR + log4jConfigurationDir;
    +            }
    +        } else {
    +            log4jConfigurationDir = stormHome + Utils.FILE_PATH_SEPARATOR + "log4j2";
    +        }
    +        
    +
    +        if (Utils.IS_ON_WINDOWS && !log4jConfigurationDir.startsWith("file:")) {
    +            log4jConfigurationDir = "file:///" + log4jConfigurationDir;
    +        }
    +        return log4jConfigurationDir + Utils.FILE_PATH_SEPARATOR + "worker.xml";
    +    }
    +    
    +    /**
    +     * Get parameters for the class path of the worker process.  Also used by the
    +     * log Writer
    +     * @param stormRoot the root dist dir for the topology
    +     * @return the classpath for the topology as command line arguments.
    +     * @throws IOException on any error.
    +     */
    +    private List<String> getClassPathParams(final String stormRoot) throws IOException {
    +        final String stormJar = ConfigUtils.supervisorStormJarPath(stormRoot);
    +        final StormTopology stormTopology = ConfigUtils.readSupervisorTopology(_conf, _topologyId, _ops);
    +        final List<String> dependencyLocations = new ArrayList<>();
    +        if (stormTopology.get_dependency_jars() != null) {
    +            for (String dependency : stormTopology.get_dependency_jars()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +
    +        if (stormTopology.get_dependency_artifacts() != null) {
    +            for (String dependency : stormTopology.get_dependency_artifacts()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +        final String workerClassPath = getWorkerClassPath(stormJar, dependencyLocations);
    +        
    +        List<String> classPathParams = new ArrayList<>();
    +        classPathParams.add("-cp");
    +        classPathParams.add(workerClassPath);
    +        return classPathParams;
    +    }
    +    
    +    /**
    +     * Get a set of java properties that are common to both the log writer and the worker processes.
    +     * These are mostly system properties that are used by logging.
    +     * @return a list of command line options
    +     */
    +    private List<String> getCommonParams() {
    +        final String stormHome = ConfigUtils.concatIfNotNull(System.getProperty("storm.home"));
    +        final String workersArtifacts = ConfigUtils.workerArtifactsRoot(_conf);
    +        String stormLogDir = ConfigUtils.getLogDir();
    +        String log4jConfigurationFile = getWorkerLoggingConfigFile(stormHome);
    +        
    +        List<String> commonParams = new ArrayList<>();
    +        commonParams.add("-Dlogfile.name=worker.log");
    +        commonParams.add("-Dstorm.home=" + stormHome);
    +        commonParams.add("-Dworkers.artifacts=" + workersArtifacts);
    +        commonParams.add("-Dstorm.id=" + _topologyId);
    +        commonParams.add("-Dworker.id=" + _workerId);
    +        commonParams.add("-Dworker.port=" + _port);
    +        commonParams.add("-Dstorm.log.dir=" + stormLogDir);
    +        commonParams.add("-Dlog4j.configurationFile=" + log4jConfigurationFile);
    +        commonParams.add("-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector");
    +        return commonParams;
    +    }
    +    
    +    private int getMemOnHeap(WorkerResources resources) {
    +        int memOnheap = 0;
    +        if (resources != null && resources.is_set_mem_on_heap() && 
    +                resources.get_mem_on_heap() > 0) {
    +            memOnheap = (int) Math.ceil(resources.get_mem_on_heap());
    +        } else {
    +            // set the default heap memory size for supervisor-test
    +            memOnheap = Utils.getInt(_topoConf.get(Config.WORKER_HEAP_MEMORY_MB), 768);
    +        }
    +        return memOnheap;
    +    }
    +    
    +    private List<String> getWorkerProfilerChildOpts(int memOnheap) {
    +        List<String> workerProfilerChildopts = new ArrayList<>();
    +        if (Utils.getBoolean(_conf.get(Config.WORKER_PROFILER_ENABLED), false)) {
    +            workerProfilerChildopts = substituteChildopts(_conf.get(Config.WORKER_PROFILER_CHILDOPTS), memOnheap);
    +        }
    +        return workerProfilerChildopts;
    +    }
    +    
    +    /**
    +     * a or b the first one that is not null
    +     * @param a something
    +     * @param b something else
    +     * @return a or b the first one that is not null
    +     */
    +    private <V> V OR(V a, V b) {
    +        return a == null ? b : a;
    +    }
    +    
    +    protected String javaCmd(String cmd) {
    +        String ret = null;
    +        String javaHome = System.getenv().get("JAVA_HOME");
    +        if (StringUtils.isNotBlank(javaHome)) {
    +            ret = javaHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR + cmd;
    +        } else {
    +            ret = cmd;
    +        }
    +        return ret;
    +    }
    +    
    +    /**
    +     * Create the command to launch the worker process
    +     * @param memOnheap the on heap memory for the worker
    +     * @param stormRoot the root dist dir for the topology
    +     * @param jlp java library path for the topology
    +     * @return the command to run
    +     * @throws IOException on any error.
    +     */
    +    private List<String> mkLaunchCommand(final int memOnheap, final String stormRoot,
    +            final String jlp) throws IOException {
    +        final String javaCmd = javaCmd("java");
    +        final String stormOptions = ConfigUtils.concatIfNotNull(System.getProperty("storm.options"));
    +        final String stormConfFile = ConfigUtils.concatIfNotNull(System.getProperty("storm.conf.file"));
    +        final String workerTmpDir = ConfigUtils.workerTmpRoot(_conf, _workerId);
    +        
    +        List<String> classPathParams = getClassPathParams(stormRoot);
    +        List<String> commonParams = getCommonParams();
    +        
    +        List<String> commandList = new ArrayList<>();
    +        //Log Writer Command...
    +        commandList.add(javaCmd);
    +        commandList.addAll(classPathParams);
    +        commandList.addAll(substituteChildopts(_topoConf.get(Config.TOPOLOGY_WORKER_LOGWRITER_CHILDOPTS)));
    +        commandList.addAll(commonParams);
    +        commandList.add("org.apache.storm.LogWriter"); //The LogWriter in turn launches the actual worker.
    +
    +        //Worker Command...
    +        commandList.add(javaCmd);
    +        commandList.add("-server");
    +        commandList.addAll(commonParams);
    +        commandList.addAll(substituteChildopts(_conf.get(Config.WORKER_CHILDOPTS), memOnheap));
    +        commandList.addAll(substituteChildopts(_topoConf.get(Config.TOPOLOGY_WORKER_CHILDOPTS), memOnheap));
    +        commandList.addAll(substituteChildopts(OR(
    +                _topoConf.get(Config.TOPOLOGY_WORKER_GC_CHILDOPTS),
    +                _conf.get(Config.WORKER_GC_CHILDOPTS)), memOnheap));
    +        commandList.addAll(getWorkerProfilerChildOpts(memOnheap));
    +        commandList.add("-Djava.library.path=" + jlp);
    +        commandList.add("-Dstorm.conf.file=" + stormConfFile);
    +        commandList.add("-Dstorm.options=" + stormOptions);
    +        commandList.add("-Djava.io.tmpdir=" + workerTmpDir);
    +        commandList.add("-Dlogging.sensitivity=" + OR((String) _topoConf.get(Config.TOPOLOGY_LOGGING_SENSITIVITY), "S3"));
    +        commandList.addAll(classPathParams);
    +        commandList.add("org.apache.storm.daemon.worker");
    +        commandList.add(_topologyId);
    +        commandList.add(_supervisorId);
    +        commandList.add(String.valueOf(_port));
    +        commandList.add(_workerId);
    +        
    +        return commandList;
    +    }
    +    
    +    @Override
    +    public void launch() throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot launch a container recovered with just a worker id");
    +        }
    +        LOG.info("Launching worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                _supervisorId, _port, _workerId);
    +        String logPrefix = "Worker Process " + _workerId;
    +        ProcessExitCallback processExitCallback = new ProcessExitCallback(logPrefix);
    +        _exitedEarly = false;
    +        
    +        final WorkerResources resources = _assignment.get_resources();
    +        final int memOnheap = getMemOnHeap(resources);
    +        final String stormRoot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +        final String jlp = jlp(stormRoot, _conf);
    +        
    +        List<String> commandList = mkLaunchCommand(memOnheap, stormRoot, jlp);
    +
    +        Map<String, String> topEnvironment = new HashMap<String, String>();
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> environment = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (environment != null) {
    +            topEnvironment.putAll(environment);
    +        }
    +        topEnvironment.put("LD_LIBRARY_PATH", jlp);
    +
    +        // {"cpu" cpu "memory" (+ mem-onheap mem-offheap (int (Math/ceil (conf
    --- End diff --
    
    This was here before and I think it is worth saving.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79032969
  
    --- 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 --
    
    But the state is WaitingForBlobLocalization.  That is how this function is called.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77421416
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    --- End diff --
    
    OK


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77418874
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,420 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + * TODO once we have replaced the original supervisor merge this with
    + * Localizer and optimize them
    + */
    +public class AsyncLocalizer implements ILocalizer, Shutdownable {
    +    /**
    +     * A future that has already completed.
    +     */
    +    private static class AllDoneFuture implements Future<Void> {
    +
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isCancelled() {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isDone() {
    +            return true;
    +        }
    +
    +        @Override
    +        public Void get() {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void get(long timeout, TimeUnit unit) {
    +            return null;
    +        }
    +
    +    }
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizer.class);
    +
    +    private final Localizer _localizer;
    +    private final ExecutorService _execService;
    +    private final boolean _isLocalMode;
    +    private final Map<String, Object> _conf;
    +    private final Map<String, LocalDownloadedResource> _basicPending;
    +    private final Map<String, LocalDownloadedResource> _blobPending;
    +    private final AdvancedFSOps _fsOps;
    +
    +    private class DownloadBaseBlobsDistributed implements Callable<Void> {
    +        private final String _topologyId;
    +        
    +        public DownloadBaseBlobsDistributed(String topologyId) {
    +            this._topologyId = topologyId;
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            File sr = new File(stormroot);
    +            if (sr.exists()) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    Utils.forceDelete(stormroot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            try {
    +                String stormJarKey = ConfigUtils.masterStormJarKey(_topologyId);
    +                String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +                String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +                String jarPath = ConfigUtils.supervisorStormJarPath(tmproot);
    +                String codePath = ConfigUtils.supervisorStormCodePath(tmproot);
    +                String confPath = ConfigUtils.supervisorStormConfPath(tmproot);
    +                FileUtils.forceMkdir(new File(tmproot));
    +                _fsOps.restrictDirectoryPermissions(tmproot);
    +                ClientBlobStore blobStore = Utils.getClientBlobStoreForSupervisor(_conf);
    +                try {
    +                    Utils.downloadResourcesAsSupervisor(stormJarKey, jarPath, blobStore);
    +                    Utils.downloadResourcesAsSupervisor(stormCodeKey, codePath, blobStore);
    +                    Utils.downloadResourcesAsSupervisor(stormConfKey, confPath, blobStore);
    +                } finally {
    +                    blobStore.shutdown();
    +                }
    +                Utils.extractDirFromJar(jarPath, ConfigUtils.RESOURCES_SUBDIR, tmproot);
    +                _fsOps.moveDirectoryPreferAtomic(new File(tmproot), new File(stormroot));
    +                SupervisorUtils.setupStormCodeDir(_conf, ConfigUtils.readSupervisorStormConf(_conf, _topologyId), stormroot);
    +                deleteAll = false;
    +            } finally {
    +                if (deleteAll) {
    +                    LOG.info("Failed to download basic resources for topology-id {}", _topologyId);
    +                    Utils.forceDelete(tmproot);
    +                    Utils.forceDelete(stormroot);
    +                }
    +            }
    +            return null;
    +        }
    +    }
    +    
    +    private class DownloadBaseBlobsLocal implements Callable<Void> {
    +        private final String _topologyId;
    +        
    +        public DownloadBaseBlobsLocal(String topologyId) {
    +            this._topologyId = topologyId;
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            File sr = new File(stormroot);
    +            if (sr.exists()) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    Utils.forceDelete(stormroot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            try {
    +                BlobStore blobStore = Utils.getNimbusBlobStore(_conf, null, null);
    +                FileOutputStream codeOutStream = null;
    +                FileOutputStream confOutStream = null;
    +                try {
    +                    FileUtils.forceMkdir(new File(tmproot));
    +                    String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +                    String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +                    String codePath = ConfigUtils.supervisorStormCodePath(tmproot);
    +                    String confPath = ConfigUtils.supervisorStormConfPath(tmproot);
    +                    codeOutStream = new FileOutputStream(codePath);
    +                    blobStore.readBlobTo(stormCodeKey, codeOutStream, null);
    +                    confOutStream = new FileOutputStream(confPath);
    +                    blobStore.readBlobTo(stormConfKey, confOutStream, null);
    +                } finally {
    +                    if (codeOutStream != null)
    +                        codeOutStream.close();
    +                    if (confOutStream != null)
    +                        codeOutStream.close();
    +                    blobStore.shutdown();
    +                }
    +
    +                ClassLoader classloader = Thread.currentThread().getContextClassLoader();
    +                String resourcesJar = AsyncLocalizer.resourcesJar();
    +                URL url = classloader.getResource(ConfigUtils.RESOURCES_SUBDIR);
    +
    +                String targetDir = tmproot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +
    +                if (resourcesJar != null) {
    +                    LOG.info("Extracting resources from jar at {} to {}", resourcesJar, targetDir);
    +                    Utils.extractDirFromJar(resourcesJar, ConfigUtils.RESOURCES_SUBDIR, stormroot);
    +                } else if (url != null) {
    +                    LOG.info("Copying resources at {} to {} ", url.toString(), targetDir);
    +                    if (url.getProtocol() == "jar") {
    +                        JarURLConnection urlConnection = (JarURLConnection) url.openConnection();
    +                        Utils.extractDirFromJar(urlConnection.getJarFileURL().getFile(), ConfigUtils.RESOURCES_SUBDIR, stormroot);
    +                    } else {
    +                        FileUtils.copyDirectory(new File(url.getFile()), (new File(targetDir)));
    +                    }
    +                }
    +                _fsOps.moveDirectoryPreferAtomic(new File(tmproot), new File(stormroot));
    +                SupervisorUtils.setupStormCodeDir(_conf, ConfigUtils.readSupervisorStormConf(_conf, _topologyId), stormroot);
    +                deleteAll = false;
    +            } finally {
    +                if (deleteAll) {
    +                    LOG.info("Failed to download basic resources for topology-id {}", _topologyId);
    +                    Utils.forceDelete(tmproot);
    +                    Utils.forceDelete(stormroot);
    +                }
    +            }
    +            return null;
    +        }
    +    }
    +    
    +    private class DownloadBlobs implements Callable<Void> {
    +        private final String _topologyId;
    +
    +        public DownloadBlobs(String topologyId) {
    +            _topologyId = topologyId;
    +        }
    +
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            Map<String, Object> stormConf = ConfigUtils.readSupervisorStormConf(_conf, _topologyId);
    +                
    +            @SuppressWarnings("unchecked")
    +            Map<String, Map<String, Object>> blobstoreMap = (Map<String, Map<String, Object>>) stormConf.get(Config.TOPOLOGY_BLOBSTORE_MAP);
    +            String user = (String) stormConf.get(Config.TOPOLOGY_SUBMITTER_USER);
    +            String topoName = (String) stormConf.get(Config.TOPOLOGY_NAME);
    +
    +            List<LocalResource> localResourceList = new ArrayList<>();
    +            if (blobstoreMap != null) {
    +                List<LocalResource> tmp = SupervisorUtils.blobstoreMapToLocalresources(blobstoreMap);
    +                if (tmp != null) {
    +                    localResourceList.addAll(tmp);
    +                }
    +            }
    +
    +            StormTopology stormCode = ConfigUtils.readSupervisorTopology(_conf, _topologyId, _fsOps);
    +            List<String> dependencies = new ArrayList<>();
    +            if (stormCode.is_set_dependency_jars()) {
    +                dependencies.addAll(stormCode.get_dependency_jars());
    +            }
    +            if (stormCode.is_set_dependency_artifacts()) {
    +                dependencies.addAll(stormCode.get_dependency_artifacts());
    +            }
    +            for (String dependency : dependencies) {
    +                localResourceList.add(new LocalResource(dependency, false));
    +            }
    +            
    +            if (!localResourceList.isEmpty()) {
    +                File userDir = _localizer.getLocalUserFileCacheDir(user);
    +                if (!userDir.exists()) {
    +                    FileUtils.forceMkdir(userDir);
    +                }
    +                List<LocalizedResource> localizedResources = _localizer.getBlobs(localResourceList, user, topoName, userDir);
    +                _fsOps.setupBlobPermissions(userDir, user);
    +                for (LocalizedResource localizedResource : localizedResources) {
    +                    File rsrcFilePath = new File(localizedResource.getFilePath());
    +                    String keyName = rsrcFilePath.getName();
    +                    String blobSymlinkTargetName = new File(localizedResource.getCurrentSymlinkPath()).getName();
    +
    +                    String symlinkName = null;
    +                    Map<String, Object> blobInfo = blobstoreMap.get(keyName);
    +                    if (blobInfo != null && blobInfo.containsKey("localname")) {
    +                        symlinkName = (String) blobInfo.get("localname");
    +                    } else {
    +                        symlinkName = keyName;
    +                    }
    +                    Utils.forceCreateSymlink(stormroot, rsrcFilePath.getParent(), symlinkName, blobSymlinkTargetName);
    +                }
    +            }
    +
    +            return null;
    +        }
    +    }
    +    
    +    public AsyncLocalizer(Map<String, Object> conf, Localizer localizer) {
    +        _conf = conf;
    +        _isLocalMode = ConfigUtils.isLocalMode(conf);
    +        _localizer = localizer;
    +        _execService = Executors.newFixedThreadPool(1,  
    +                new ThreadFactoryBuilder()
    +                .setNameFormat("Async Localizer")
    +                .build());
    +        _basicPending = new HashMap<>();
    +        _blobPending = new HashMap<>();
    +        _fsOps = AdvancedFSOps.make(_conf);
    +    }
    +
    +    @Override
    +    public synchronized Future<Void> requestDownloadBaseTopologyBlobs(final String topologyId, final int port) {
    +        LocalDownloadedResource localResource = _basicPending.get(topologyId);
    +        if (localResource == null) {
    +            Callable<Void> c;
    +            if (_isLocalMode) {
    +                c = new DownloadBaseBlobsLocal(topologyId);
    +            } else {
    +                c = new DownloadBaseBlobsDistributed(topologyId);
    +            }
    +            localResource = new LocalDownloadedResource(_execService.submit(c));
    +            _basicPending.put(topologyId, localResource);
    +        }
    +        return localResource.reserve(port);
    +    }
    +
    +    private static String resourcesJar() throws IOException {
    +        String path = Utils.currentClasspath();
    +        if (path == null) {
    +            return null;
    +        }
    +        String[] paths = path.split(File.pathSeparator);
    +        List<String> jarPaths = new ArrayList<String>();
    +        for (String s : paths) {
    +            if (s.endsWith(".jar")) {
    +                jarPaths.add(s);
    +            }
    +        }
    +
    +        List<String> rtn = new ArrayList<String>();
    +        int size = jarPaths.size();
    +        for (int i = 0; i < size; i++) {
    +            if (Utils.zipDoesContainDir(jarPaths.get(i), ConfigUtils.RESOURCES_SUBDIR)) {
    +                rtn.add(jarPaths.get(i));
    +            }
    +        }
    +        if (rtn.size() == 0)
    +            return null;
    +
    +        return rtn.get(0);
    +    }
    +    
    +    @Override
    +    public synchronized void recoverRunningTopology(String topologyId, int port) {
    +        LocalDownloadedResource localResource = _basicPending.get(topologyId);
    +        if (localResource == null) {
    +            localResource = new LocalDownloadedResource(new AllDoneFuture());
    +            _basicPending.put(topologyId, localResource);
    +        }
    +        localResource = _blobPending.get(topologyId);
    +        if (localResource == null) {
    +            localResource = new LocalDownloadedResource(new AllDoneFuture());
    +            _blobPending.put(topologyId, localResource);
    +        }
    +    }
    +    
    +    @Override
    +    public synchronized Future<Void> requestDownloadTopologyBlobs(String topologyId, int port) {
    +        LocalDownloadedResource localResource = _blobPending.get(topologyId);
    +        if (localResource == null) {
    +            Callable<Void> c = new DownloadBlobs(topologyId);
    +            localResource = new LocalDownloadedResource(_execService.submit(c));
    +            _blobPending.put(topologyId, localResource);
    +        }
    +        return localResource.reserve(port);
    +    }
    +
    +    @Override
    +    public synchronized void releaseSlotFor(String topologyId, int port) throws IOException {
    +        LOG.warn("Releaseing slot for {} {}", topologyId, port);
    --- End diff --
    
    "Releasing"


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77414253
  
    --- 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);
    +            }
    +            if (container != null) {
    +                sb.append(" container: ");
    +                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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WAITING_FOR_WORKER_START) {
    +                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;
    +                    } else 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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), 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.get_topology_id(), 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);
    +        
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    --- End diff --
    
    Might want to add the reasoning: It is because properly cancelling the other thread's downloads while other workers might need the same files adds a ton of complexity for an uncommon case.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79039785
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,328 @@
    +/**
    + * 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.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 java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +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.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this.superConf = supervisor.getConf();
    +        this.stormClusterState = supervisor.getStormClusterState();
    +        this.syncSupEventManager = supervisor.getEventManger();
    +        this.assignmentVersions = new AtomicReference<>(new HashMap<>());
    +        this.assignmentId = supervisor.getAssignmentId();
    +        this.iSuper = supervisor.getiSupervisor();
    +        this.localizer = supervisor.getAsyncLocalizer();
    +        this.host = supervisor.getHostName();
    +        this.localState = supervisor.getLocalState();
    +        this.clusterState = supervisor.getStormClusterState();
    +        this.cachedAssignments = supervisor.getCurrAssignment();
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, supervisor.getSharedContext());
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +        
    +        try {
    +            Collection<String> workers = SupervisorUtils.supervisorWorkerIds(superConf);
    +            for (Slot slot: slots.values()) {
    +                String workerId = slot.getWorkerId();
    +                if (workerId != null) {
    +                    workers.remove(workerId);
    +                }
    +            }
    +            if (!workers.isEmpty()) {
    +                supervisor.killWorkers(workers, launcher);
    +            }
    +        } catch (Exception e) {
    +            LOG.warn("Error trying to clean up old workers", e);
    +        }
    +
    +        //All the slots/assignments should be recovered now, so we can clean up anything that we don't expect to be here
    +        try {
    +            localizer.cleanupUnusedTopologies();
    +        } catch (Exception e) {
    +            LOG.warn("Error trying to clean up old topologies", e);
    +        }
    +        
    +        for (Slot slot: slots.values()) {
    +            slot.start();
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        return new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    +                            if (actions == null) {
    +                                actions = new HashSet<>();
    +                                filtered.put(port.intValue(), actions);
    +                            }
    +                            actions.add(new TopoProfileAction(topoId, req));
    +                        }
    +                    }
    +                }
    +            }
    +            
    +            for (Integer port: allPorts) {
    +                Slot slot = slots.get(port);
    +                if (slot == null) {
    +                    slot = mkSlot(port);
    +                    slots.put(port, slot);
    +                    slot.start();
    +                }
    +                slot.setNewAssignment(allAssignments.get(port));
    +                slot.addProfilerActions(filtered.get(port));
    +            }
    +            
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +    }
    +    
    +    protected Map<String, VersionedData<Assignment>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +            Map<String, VersionedData<Assignment>> localAssignmentVersion, Runnable callback) throws Exception {
    +        Map<String, VersionedData<Assignment>> updateAssignmentVersion = new HashMap<>();
    +        for (String topoId : topoIds) {
    +            Integer recordedVersion = -1;
    +            Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +            VersionedData<Assignment> locAssignment = localAssignmentVersion.get(topoId);
    +            if (locAssignment != null) {
    +                recordedVersion = locAssignment.getVersion();
    +            }
    +            if (version == null) {
    +                // ignore
    +            } else if (version == recordedVersion) {
    +                updateAssignmentVersion.put(topoId, locAssignment);
    +            } else {
    +                VersionedData<Assignment> assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                updateAssignmentVersion.put(topoId, assignmentVersion);
    +            }
    +        }
    +        return updateAssignmentVersion;
    +    }
    +    
    +    protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +        Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +        for (String stormId : stormIds) {
    +            List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +            ret.put(stormId, profileRequests);
    +        }
    +        return ret;
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readAssignments(Map<String, VersionedData<Assignment>> assignmentsSnapshot,
    +            String assignmentId, AtomicInteger retries) {
    --- End diff --
    
    Oh that was when we had a SupervisorData object that was separate from the Supervisor.  I'll update it to not pass it in.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @HeartSaVioR @harshach Once this goes in I would be happy to make a version for 1.x
    
    In fact we are in the process of pulling it back for the version we use internally, and I was able to keep the old supervisor in place as a backup option.
    
    As for the regression I will try and find the original code and see where I made a mistake.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76169008
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Supervisor.java ---
    @@ -17,135 +17,541 @@
      */
     package org.apache.storm.daemon.supervisor;
     
    +import java.io.File;
    +import java.io.IOException;
    +import java.net.UnknownHostException;
    +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.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
     import org.apache.commons.io.FileUtils;
     import org.apache.storm.Config;
     import org.apache.storm.StormTimer;
    -import org.apache.storm.daemon.supervisor.timer.RunProfilerActions;
    +import org.apache.storm.cluster.ClusterStateContext;
    +import org.apache.storm.cluster.ClusterUtils;
    +import org.apache.storm.cluster.DaemonType;
    +import org.apache.storm.cluster.IStateStorage;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.daemon.DaemonCommon;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
     import org.apache.storm.daemon.supervisor.timer.SupervisorHealthCheck;
     import org.apache.storm.daemon.supervisor.timer.SupervisorHeartbeat;
     import org.apache.storm.daemon.supervisor.timer.UpdateBlobs;
    +import org.apache.storm.event.EventManager;
     import org.apache.storm.event.EventManagerImp;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.AsyncLocalizer;
    +import org.apache.storm.localizer.ILocalizer;
     import org.apache.storm.localizer.Localizer;
     import org.apache.storm.messaging.IContext;
     import org.apache.storm.metric.StormMetricsRegistry;
     import org.apache.storm.scheduler.ISupervisor;
     import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
     import org.apache.storm.utils.Utils;
     import org.apache.storm.utils.VersionInfo;
    +import org.apache.zookeeper.data.ACL;
    +import org.eclipse.jetty.util.ConcurrentHashSet;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -import java.io.File;
    -import java.io.InterruptedIOException;
    -import java.util.Collection;
    -import java.util.Map;
    -import java.util.Set;
    -import java.util.concurrent.Callable;
    +public class Supervisor implements DaemonCommon, AutoCloseable {
    +    
    +    public class ReadStateThread implements Runnable, AutoCloseable {
    --- End diff --
    
    This is big enough that I wonder if it should be its own class. I don't see that it's taking advantage of the special access inner classes have to any extent, and we already have a supervisor package it could comfortably live in.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77416281
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    --- End diff --
    
    There already is one inside readAssignments.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76472894
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,437 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = -1;
    +        _assignment = null;
    +        _workerId = workerId;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    --- End diff --
    
    debug


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76853245
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,749 @@
    +/**
    + * 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,
    +        WATING_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 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 monitorFreqMs,
    +                ContainerLauncher containerLauncher, String host, int port,
    +                ISupervisor iSupervisor, LocalState localState) {
    +            this.localizer = localizer;
    +            this.hbTimeoutMs = hbTimeoutMs;
    +            this.firstHbTimeoutMs = firstHbTimeoutMs;
    +            this.containerLauncher = containerLauncher;
    +            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 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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WATING_FOR_WORKER_START) {
    +                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 equivilant(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;
    +                    } else 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 WATING_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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), staticState.port);
    +        }
    +        Time.sleep(1000);
    +        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(1000);
    +        
    +        //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.get_topology_id(), 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);
    +        
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivilant(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WATING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +        
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WATING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    --- End diff --
    
    @revans2 agree on too many configs . But can we add atleast some comment on why the value 120000 being used.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78949081
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ContainerLauncher.java ---
    @@ -0,0 +1,103 @@
    +/**
    + * 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.Map;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Launches containers
    + */
    +public abstract class ContainerLauncher {
    +    private static final Logger LOG = LoggerFactory.getLogger(ContainerLauncher.class);
    +    
    +    /**
    +     * Factory to create the right container launcher 
    +     * for the config and the environment.
    +     * @param conf the config
    +     * @param supervisorId the ID of the supervisor
    +     * @param sharedContext Used in local mode to let workers talk together without netty
    +     * @return the proper container launcher
    +     * @throws IOException on any error
    +     */
    +    public static ContainerLauncher make(Map<String, Object> conf, String supervisorId, IContext sharedContext) throws IOException {
    +        if (ConfigUtils.isLocalMode(conf)) {
    +            return new LocalContainerLauncher(conf, supervisorId, sharedContext);
    +        }
    +        
    +        ResourceIsolationInterface resourceIsolationManager = null;
    +        if (Utils.getBoolean(conf.get(Config.STORM_RESOURCE_ISOLATION_PLUGIN_ENABLE), false)) {
    +            resourceIsolationManager = Utils.newInstance((String) conf.get(Config.STORM_RESOURCE_ISOLATION_PLUGIN));
    +            resourceIsolationManager.prepare(conf);
    +            LOG.info("Using resource isolation plugin {} {}", conf.get(Config.STORM_RESOURCE_ISOLATION_PLUGIN), resourceIsolationManager);
    +        }
    +
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new RunAsUserContainerLauncher(conf, supervisorId, resourceIsolationManager);
    +        }
    +        return new BasicContainerLauncher(conf, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    protected ContainerLauncher() {
    +        //Empty
    +    }
    +
    +    /**
    +     * Launch a container in a given slot
    +     * @param port the port to run this on
    +     * @param assignment what to launch
    +     * @param state the current state of the supervisor
    +     * @return The container that can be used to manager the processes.
    +     * @throws IOException on any error 
    +     */
    +    public abstract Container launchContainer(int port, LocalAssignment assignment, LocalState state) throws IOException;
    +    
    +    /**
    +     * Recover a container for a running process
    +     * @param port the port the assignment is running on
    +     * @param assignment the assignment that was launched
    +     * @param state the current state of the supervisor
    +     * @return The container that can be used to manage the processes.
    +     * @throws IOException on any error
    +     */
    +    public abstract Container recoverContainer(int port, LocalAssignment assignment, LocalState state) throws IOException;
    --- End diff --
    
    I'm wondering if this would be clearer if this returned Optional<Container>? It seems like the implementations will return null if the container couldn't be recovered.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79030835
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected final String _stormHome = System.getProperty("storm.home");
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +    
    +    /**
    +     * Create a new BasicContainer
    +     * @param type the type of container being made.
    +     * @param conf the supervisor config
    +     * @param supervisorId the ID of the supervisor this is a part of.
    +     * @param port the port the container is on.  Should be <= 0 if only a partial recovery
    +     * @param assignment the assignment for this container. Should be null if only a partial recovery.
    +     * @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used.
    +     * @param localState the local state of the supervisor.  May be null if partial recovery
    +     * @param workerId the id of the worker to use.  Must not be null if doing a partial recovery.
    +     * @param ops file system operations (mostly for testing) if null a new one is made
    +     * @param topoConf the config of the topology (mostly for testing) if null 
    +     * and not a partial recovery the real conf is read.
    +     * @param profileCmd the command to use when profiling (used for testing)
    +     */
    +    public BasicContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int port,
    +            LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
    +            LocalState localState, String workerId, Map<String, Object> topoConf, 
    +            AdvancedFSOps ops, String profileCmd) throws IOException {
    +        super(type, conf, supervisorId, port, assignment, resourceIsolationManager, workerId, topoConf, ops);
    +        assert(localState != null);
    +        _localState = localState;
    +
    +        if (type.isRecovery() && !type.isOnlyKillable()) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                LOG.info("Recovered Worker {}", wid);
    +                _workerId = wid;
    +            }
    +        } else if (_workerId == null){
    +            createNewWorkerId();
    +        }
    +
    +        if (profileCmd == null) {
    +            profileCmd = _stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                    + conf.get(Config.WORKER_PROFILER_COMMAND);
    +        }
    +        _profileCmd = profileCmd;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        _type.assertFull();
    +        assert(_workerId == null);
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            removeWorkersOn(workerToPort, _port);
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +            LOG.info("Created Worker ID {}", _workerId);
    +        }
    +    }
    +
    +    private static void removeWorkersOn(Map<String, Integer> workerToPort, int _port) {
    +        for (Iterator<Entry<String, Integer>> i = workerToPort.entrySet().iterator(); i.hasNext();) {
    +            Entry<String, Integer> found = i.next();
    +            if (_port == found.getValue().intValue()) {
    +                LOG.warn("Deleting worker {} from state", found.getKey());
    +                i.remove();
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUpForRestart() throws IOException {
    +        String origWorkerId = _workerId;
    +        super.cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(origWorkerId);
    +            removeWorkersOn(workersToPort, _port);
    +            _localState.setApprovedWorkers(workersToPort);
    +            LOG.info("Removed Worker ID {}", origWorkerId);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        _type.assertFull();
    +        //We are launching it now...
    +        _type = ContainerType.LAUNCH;
    +        createNewWorkerId();
    +        setup();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        _type.assertFull();
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        _type.assertFull();
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = _ops.slurpString(new File(str)).trim();
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        if (command.size() > 0) {
    +            return runProfilingCommand(command, env, logPrefix, targetFile);
    +        }
    +        LOG.warn("PROFILING REQUEST NOT SUPPORTED {} IGNORED...", request);
    +        return true;
    +    }
    +
    +    /**
    +     * Get the command to run when doing profiling
    +     * @param action the profiling action to perform
    +     * @param stop if this is meant to stop the profiling or start it
    +     * @param workerPid the PID of the process to profile
    +     * @param targetDir the current working directory of the worker process
    +     * @return the command to run for profiling.
    +     */
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        switch(action) {
    +            case JMAP_DUMP:
    +                return jmapDumpCmd(workerPid, targetDir);
    +            case JSTACK_DUMP:
    +                return jstackDumpCmd(workerPid, targetDir);
    +            case JPROFILE_DUMP:
    +                return jprofileDump(workerPid, targetDir);
    +            case JVM_RESTART:
    +                return jprofileJvmRestart(workerPid);
    +            case JPROFILE_STOP:
    +                if (stop) {
    +                    return jprofileStop(workerPid, targetDir);
    +                }
    +                return jprofileStart(workerPid);
    +            default:
    +                return Lists.newArrayList();
    +        }
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +
    +    /**
    +     * Compute the java.library.path that should be used for the worker.
    +     * This helps it to load JNI libraries that are packaged in the uber jar.
    +     * @param stormRoot the root directory of the worker process
    +     * @param conf the config for the supervisor.
    +     * @return the java.library.path/LD_LIBRARY_PATH to use so native libraries load correctly.
    +     */
    +    protected String javaLibraryPath(String stormRoot, Map<String, Object> conf) {
    +        String resourceRoot = stormRoot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +        String os = System.getProperty("os.name").replaceAll("\\s+", "_");
    +        String arch = System.getProperty("os.arch");
    +        String archResourceRoot = resourceRoot + Utils.FILE_PATH_SEPARATOR + os + "-" + arch;
    +        String ret = CPJ.join(archResourceRoot, resourceRoot,
    +                conf.get(Config.JAVA_LIBRARY_PATH));
    +        return ret;
    +    }
    +
    +    /**
    +     * Returns a collection of jar file names found under the given directory.
    +     * @param dir the directory to search
    +     * @return the jar file names
    +     */
    +    protected List<String> getFullJars(File dir) {
    +        File[] files = dir.listFiles(jarFilter);
    +
    +        if (files == null) {
    +            return Collections.emptyList();
    +        }
    +
    +        return Arrays.stream(files).map(f -> f.getAbsolutePath())
    +                .collect(Collectors.toList());
    +    }
    +    
    +    protected List<String> frameworkClasspath() {
    +        File stormLibDir = new File(_stormHome, "lib");
    +        String stormConfDir =
    +                System.getenv("STORM_CONF_DIR") != null ?
    +                System.getenv("STORM_CONF_DIR") :
    +                new File(_stormHome, "conf").getAbsolutePath();
    +        File stormExtlibDir = new File(_stormHome, "extlib");
    +        String extcp = System.getenv("STORM_EXT_CLASSPATH");
    +        List<String> pathElements = new LinkedList<>();
    +        pathElements.addAll(getFullJars(stormLibDir));
    +        pathElements.addAll(getFullJars(stormExtlibDir));
    +        pathElements.add(extcp);
    +        pathElements.add(stormConfDir);
    +
    +        return pathElements;
    +    }
    +    
    +    @SuppressWarnings("unchecked")
    +    private List<String> asStringList(Object o) {
    +        if (o instanceof String) {
    +            return Arrays.asList((String)o);
    +        } else if (o instanceof List) {
    +            return (List<String>)o;
    +        }
    +        return Collections.EMPTY_LIST;
    +    }
    +    
    +    /**
    +     * Compute the classpath for the worker process
    +     * @param stormJar the topology jar
    +     * @param dependencyLocations any dependencies from the topology
    +     * @return the full classpath
    +     */
    +    protected String getWorkerClassPath(String stormJar, List<String> dependencyLocations) {
    +        List<String> workercp = new ArrayList<>();
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH_BEGINNING)));
    +        workercp.addAll(frameworkClasspath());
    +        workercp.add(stormJar);
    +        workercp.addAll(dependencyLocations);
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH)));
    +        return CPJ.join(workercp);
    +    }
    +
    +    private String substituteChildOptsInternal(String string, int memOnheap) {
    +        if (StringUtils.isNotBlank(string)) {
    +            String p = String.valueOf(_port);
    +            string = string.replace("%ID%", p);
    +            string = string.replace("%WORKER-ID%", _workerId);
    +            string = string.replace("%TOPOLOGY-ID%", _topologyId);
    +            string = string.replace("%WORKER-PORT%", p);
    +            if (memOnheap > 0) {
    +                string = string.replace("%HEAP-MEM%", String.valueOf(memOnheap));
    +            }
    +        }
    +        return string;
    +    }
    +    
    +    protected List<String> substituteChildopts(Object value) {
    +        return substituteChildopts(value, -1);
    +    }
    +
    +    protected List<String> substituteChildopts(Object value, int memOnheap) {
    +        List<String> rets = new ArrayList<>();
    +        if (value instanceof String) {
    +            String string = substituteChildOptsInternal((String) value, memOnheap);
    +            if (StringUtils.isNotBlank(string)) {
    +                String[] strings = string.split("\\s+");
    +                rets.addAll(Arrays.asList(strings));
    +            }
    +        } else if (value instanceof List) {
    +            @SuppressWarnings("unchecked")
    +            List<String> objects = (List<String>) value;
    +            for (String object : objects) {
    +                String str = substituteChildOptsInternal(object, memOnheap);
    +                if (StringUtils.isNotBlank(str)) {
    +                    rets.add(str);
    +                }
    +            }
    +        }
    +        return rets;
    +    }
    +
    +    /**
    +     * Launch the worker process (non-blocking)
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param processExitcallback
    +     *            a callback for when the process exits
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     */
    +    protected void launchWorkerProcess(List<String> command, Map<String, String> env, String logPrefix,
    +            ExitCodeCallback processExitCallback, File targetDir) throws IOException {
    +        if (_resourceIsolationManager != null) {
    +          command = _resourceIsolationManager.getLaunchCommand(_workerId, command);
    +        }
    +        SupervisorUtils.launchProcess(command, env, logPrefix, processExitCallback, targetDir);
    +    }
    +
    +    private String getWorkerLoggingConfigFile() {
    +        String log4jConfigurationDir = (String) (_conf.get(Config.STORM_LOG4J2_CONF_DIR));
    +
    +        if (StringUtils.isNotBlank(log4jConfigurationDir)) {
    +            if (!Utils.isAbsolutePath(log4jConfigurationDir)) {
    +                log4jConfigurationDir = _stormHome + Utils.FILE_PATH_SEPARATOR + log4jConfigurationDir;
    +            }
    +        } else {
    +            log4jConfigurationDir = _stormHome + Utils.FILE_PATH_SEPARATOR + "log4j2";
    +        }
    + 
    +        if (Utils.IS_ON_WINDOWS && !log4jConfigurationDir.startsWith("file:")) {
    +            log4jConfigurationDir = "file:///" + log4jConfigurationDir;
    +        }
    +        return log4jConfigurationDir + Utils.FILE_PATH_SEPARATOR + "worker.xml";
    +    }
    +    
    +    /**
    +     * Get parameters for the class path of the worker process.  Also used by the
    +     * log Writer
    +     * @param stormRoot the root dist dir for the topology
    +     * @return the classpath for the topology as command line arguments.
    +     * @throws IOException on any error.
    +     */
    +    private List<String> getClassPathParams(final String stormRoot) throws IOException {
    +        final String stormJar = ConfigUtils.supervisorStormJarPath(stormRoot);
    +        final StormTopology stormTopology = ConfigUtils.readSupervisorTopology(_conf, _topologyId, _ops);
    +        final List<String> dependencyLocations = new ArrayList<>();
    +        if (stormTopology.get_dependency_jars() != null) {
    +            for (String dependency : stormTopology.get_dependency_jars()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +
    +        if (stormTopology.get_dependency_artifacts() != null) {
    +            for (String dependency : stormTopology.get_dependency_artifacts()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +        final String workerClassPath = getWorkerClassPath(stormJar, dependencyLocations);
    +        
    +        List<String> classPathParams = new ArrayList<>();
    +        classPathParams.add("-cp");
    +        classPathParams.add(workerClassPath);
    +        return classPathParams;
    +    }
    +    
    +    /**
    +     * Get a set of java properties that are common to both the log writer and the worker processes.
    +     * These are mostly system properties that are used by logging.
    +     * @return a list of command line options
    +     */
    +    private List<String> getCommonParams() {
    +        final String workersArtifacts = ConfigUtils.workerArtifactsRoot(_conf);
    +        String stormLogDir = ConfigUtils.getLogDir();
    +        String log4jConfigurationFile = getWorkerLoggingConfigFile();
    +        
    +        List<String> commonParams = new ArrayList<>();
    +        commonParams.add("-Dlogging.sensitivity=" + OR((String) _topoConf.get(Config.TOPOLOGY_LOGGING_SENSITIVITY), "S3"));
    +        commonParams.add("-Dlogfile.name=worker.log");
    +        commonParams.add("-Dstorm.home=" + OR(_stormHome, ""));
    +        commonParams.add("-Dworkers.artifacts=" + workersArtifacts);
    +        commonParams.add("-Dstorm.id=" + _topologyId);
    +        commonParams.add("-Dworker.id=" + _workerId);
    +        commonParams.add("-Dworker.port=" + _port);
    +        commonParams.add("-Dstorm.log.dir=" + stormLogDir);
    +        commonParams.add("-Dlog4j.configurationFile=" + log4jConfigurationFile);
    +        commonParams.add("-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector");
    +        commonParams.add("-Dstorm.local.dir=" + _conf.get(Config.STORM_LOCAL_DIR));
    +        return commonParams;
    +    }
    +    
    +    private int getMemOnHeap(WorkerResources resources) {
    +        int memOnheap = 0;
    +        if (resources != null && resources.is_set_mem_on_heap() && 
    +                resources.get_mem_on_heap() > 0) {
    +            memOnheap = (int) Math.ceil(resources.get_mem_on_heap());
    +        } else {
    +            // set the default heap memory size for supervisor-test
    +            memOnheap = Utils.getInt(_topoConf.get(Config.WORKER_HEAP_MEMORY_MB), 768);
    +        }
    +        return memOnheap;
    +    }
    +    
    +    private List<String> getWorkerProfilerChildOpts(int memOnheap) {
    +        List<String> workerProfilerChildopts = new ArrayList<>();
    +        if (Utils.getBoolean(_conf.get(Config.WORKER_PROFILER_ENABLED), false)) {
    +            workerProfilerChildopts = substituteChildopts(_conf.get(Config.WORKER_PROFILER_CHILDOPTS), memOnheap);
    +        }
    +        return workerProfilerChildopts;
    +    }
    +    
    +    /**
    +     * a or b the first one that is not null
    +     * @param a something
    +     * @param b something else
    +     * @return a or b the first one that is not null
    +     */
    +    private <V> V OR(V a, V b) {
    +        return a == null ? b : a;
    +    }
    +    
    +    protected String javaCmd(String cmd) {
    +        String ret = null;
    +        String javaHome = System.getenv().get("JAVA_HOME");
    +        if (StringUtils.isNotBlank(javaHome)) {
    +            ret = javaHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR + cmd;
    +        } else {
    +            ret = cmd;
    +        }
    +        return ret;
    +    }
    +    
    +    /**
    +     * Create the command to launch the worker process
    +     * @param memOnheap the on heap memory for the worker
    +     * @param stormRoot the root dist dir for the topology
    +     * @param jlp java library path for the topology
    +     * @return the command to run
    +     * @throws IOException on any error.
    +     */
    +    private List<String> mkLaunchCommand(final int memOnheap, final String stormRoot,
    +            final String jlp) throws IOException {
    +        final String javaCmd = javaCmd("java");
    +        final String stormOptions = ConfigUtils.concatIfNotNull(System.getProperty("storm.options"));
    +        final String stormConfFile = ConfigUtils.concatIfNotNull(System.getProperty("storm.conf.file"));
    +        final String workerTmpDir = ConfigUtils.workerTmpRoot(_conf, _workerId);
    +        
    +        List<String> classPathParams = getClassPathParams(stormRoot);
    +        List<String> commonParams = getCommonParams();
    +        
    +        List<String> commandList = new ArrayList<>();
    +        //Log Writer Command...
    +        commandList.add(javaCmd);
    +        commandList.addAll(classPathParams);
    +        commandList.addAll(substituteChildopts(_topoConf.get(Config.TOPOLOGY_WORKER_LOGWRITER_CHILDOPTS)));
    +        commandList.addAll(commonParams);
    +        commandList.add("org.apache.storm.LogWriter"); //The LogWriter in turn launches the actual worker.
    +
    +        //Worker Command...
    +        commandList.add(javaCmd);
    +        commandList.add("-server");
    +        commandList.addAll(commonParams);
    +        commandList.addAll(substituteChildopts(_conf.get(Config.WORKER_CHILDOPTS), memOnheap));
    +        commandList.addAll(substituteChildopts(_topoConf.get(Config.TOPOLOGY_WORKER_CHILDOPTS), memOnheap));
    +        commandList.addAll(substituteChildopts(OR(
    +                _topoConf.get(Config.TOPOLOGY_WORKER_GC_CHILDOPTS),
    +                _conf.get(Config.WORKER_GC_CHILDOPTS)), memOnheap));
    +        commandList.addAll(getWorkerProfilerChildOpts(memOnheap));
    +        commandList.add("-Djava.library.path=" + jlp);
    +        commandList.add("-Dstorm.conf.file=" + stormConfFile);
    +        commandList.add("-Dstorm.options=" + stormOptions);
    +        commandList.add("-Djava.io.tmpdir=" + workerTmpDir);
    +        commandList.addAll(classPathParams);
    +        commandList.add("org.apache.storm.daemon.worker");
    +        commandList.add(_topologyId);
    +        commandList.add(_supervisorId);
    +        commandList.add(String.valueOf(_port));
    +        commandList.add(_workerId);
    +        
    +        return commandList;
    +    }
    +
    +    @Override
    +    public void launch() throws IOException {
    +        _type.assertFull();
    +        LOG.info("Launching worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                _supervisorId, _port, _workerId);
    +        String logPrefix = "Worker Process " + _workerId;
    +        ProcessExitCallback processExitCallback = new ProcessExitCallback(logPrefix);
    +        _exitedEarly = false;
    +        
    +        final WorkerResources resources = _assignment.get_resources();
    +        final int memOnheap = getMemOnHeap(resources);
    +        final String stormRoot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +        final String jlp = javaLibraryPath(stormRoot, _conf);
    +        
    +        List<String> commandList = mkLaunchCommand(memOnheap, stormRoot, jlp);
    +
    +        Map<String, String> topEnvironment = new HashMap<String, String>();
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> environment = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (environment != null) {
    +            topEnvironment.putAll(environment);
    +        }
    +        topEnvironment.put("LD_LIBRARY_PATH", jlp);
    --- End diff --
    
    If I have `JNI.so` and it depends on `someLibrary.so` without `LD_LIBRARY_PATH` `JNI.so` will not be able to find  `someLibrary.so` if both were shipped in the jar together.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78997835
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,422 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.net.URLDecoder;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + */
    +public class AsyncLocalizer implements ILocalizer, Shutdownable {
    +    /**
    +     * A future that has already completed.
    +     */
    +    private static class AllDoneFuture implements Future<Void> {
    +
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isCancelled() {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isDone() {
    +            return true;
    +        }
    +
    +        @Override
    +        public Void get() {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void get(long timeout, TimeUnit unit) {
    +            return null;
    +        }
    +
    +    }
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizer.class);
    +
    +    private final Localizer _localizer;
    +    private final ExecutorService _execService;
    +    private final boolean _isLocalMode;
    +    private final Map<String, Object> _conf;
    +    private final Map<String, LocalDownloadedResource> _basicPending;
    +    private final Map<String, LocalDownloadedResource> _blobPending;
    +    private final AdvancedFSOps _fsOps;
    +
    +    private class DownloadBaseBlobsDistributed implements Callable<Void> {
    +        protected final String _topologyId;
    +        protected final File _stormRoot;
    +        
    +        public DownloadBaseBlobsDistributed(String topologyId) throws IOException {
    +            _topologyId = topologyId;
    +            _stormRoot = new File(ConfigUtils.supervisorStormDistRoot(_conf, _topologyId));
    +        }
    +        
    +        protected void downloadBaseBlobs(File tmproot) throws Exception {
    +            String stormJarKey = ConfigUtils.masterStormJarKey(_topologyId);
    +            String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +            String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +            String jarPath = ConfigUtils.supervisorStormJarPath(tmproot.getAbsolutePath());
    +            String codePath = ConfigUtils.supervisorStormCodePath(tmproot.getAbsolutePath());
    +            String confPath = ConfigUtils.supervisorStormConfPath(tmproot.getAbsolutePath());
    +            _fsOps.forceMkdir(tmproot);
    +            _fsOps.restrictDirectoryPermissions(tmproot);
    +            ClientBlobStore blobStore = Utils.getClientBlobStoreForSupervisor(_conf);
    +            try {
    +                Utils.downloadResourcesAsSupervisor(stormJarKey, jarPath, blobStore);
    +                Utils.downloadResourcesAsSupervisor(stormCodeKey, codePath, blobStore);
    +                Utils.downloadResourcesAsSupervisor(stormConfKey, confPath, blobStore);
    +            } finally {
    +                blobStore.shutdown();
    +            }
    +            Utils.extractDirFromJar(jarPath, ConfigUtils.RESOURCES_SUBDIR, tmproot);
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            if (_fsOps.fileExists(_stormRoot)) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    _fsOps.deleteIfExists(_stormRoot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            File tr = new File(tmproot);
    +            try {
    +                downloadBaseBlobs(tr);
    +                _fsOps.moveDirectoryPreferAtomic(tr, _stormRoot);
    +                _fsOps.setupStormCodeDir(ConfigUtils.readSupervisorStormConf(_conf, _topologyId), _stormRoot);
    +                deleteAll = false;
    +            } finally {
    +                if (deleteAll) {
    +                    LOG.warn("Failed to download basic resources for topology-id {}", _topologyId);
    +                    _fsOps.deleteIfExists(tr);
    +                    _fsOps.deleteIfExists(_stormRoot);
    +                }
    +            }
    +            return null;
    +        }
    +    }
    +    
    +    private class DownloadBaseBlobsLocal extends DownloadBaseBlobsDistributed {
    +
    +        public DownloadBaseBlobsLocal(String topologyId) throws IOException {
    +            super(topologyId);
    +        }
    +        
    +        @Override
    +        protected void downloadBaseBlobs(File tmproot) throws Exception {
    +            _fsOps.forceMkdir(tmproot);
    +            String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +            String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +            File codePath = new File(ConfigUtils.supervisorStormCodePath(tmproot.getAbsolutePath()));
    +            File confPath = new File(ConfigUtils.supervisorStormConfPath(tmproot.getAbsolutePath()));
    +            BlobStore blobStore = Utils.getNimbusBlobStore(_conf, null);
    +            try {
    +                try (OutputStream codeOutStream = _fsOps.getOutputStream(codePath)){
    +                    blobStore.readBlobTo(stormCodeKey, codeOutStream, null);
    +                }
    +                try (OutputStream confOutStream = _fsOps.getOutputStream(confPath)) {
    +                    blobStore.readBlobTo(stormConfKey, confOutStream, null);
    +                }
    +            } finally {
    +                blobStore.shutdown();
    +            }
    +
    +            ClassLoader classloader = Thread.currentThread().getContextClassLoader();
    +            String resourcesJar = AsyncLocalizer.resourcesJar();
    +            URL url = classloader.getResource(ConfigUtils.RESOURCES_SUBDIR);
    +
    +            String targetDir = tmproot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +
    +            if (resourcesJar != null) {
    +                LOG.info("Extracting resources from jar at {} to {}", resourcesJar, targetDir);
    +                Utils.extractDirFromJar(resourcesJar, ConfigUtils.RESOURCES_SUBDIR, _stormRoot);
    +            } else if (url != null) {
    +                LOG.info("Copying resources at {} to {} ", url.toString(), targetDir);
    +                if ("jar".equals(url.getProtocol())) {
    +                    JarURLConnection urlConnection = (JarURLConnection) url.openConnection();
    +                    Utils.extractDirFromJar(urlConnection.getJarFileURL().getFile(), ConfigUtils.RESOURCES_SUBDIR, _stormRoot);
    +                } else {
    +                    _fsOps.copyDirectory(new File(url.getFile()), new File(targetDir));
    +                }
    +            }
    +        }
    +    }
    +    
    +    private class DownloadBlobs implements Callable<Void> {
    +        private final String _topologyId;
    +
    +        public DownloadBlobs(String topologyId) {
    +            _topologyId = topologyId;
    +        }
    +
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    --- End diff --
    
    You should probably consider throwing a try-catch Exception or even Throwable around these callables for logging, in case there are bugs. I think ExecutorService swallows exceptions by default.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78993008
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Supervisor.java ---
    @@ -17,135 +17,231 @@
      */
     package org.apache.storm.daemon.supervisor;
     
    +import java.io.File;
    +import java.io.IOException;
    +import java.net.UnknownHostException;
    +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 java.util.concurrent.Callable;
    +import java.util.concurrent.atomic.AtomicReference;
    +
     import org.apache.commons.io.FileUtils;
     import org.apache.storm.Config;
     import org.apache.storm.StormTimer;
    -import org.apache.storm.daemon.supervisor.timer.RunProfilerActions;
    +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.supervisor.timer.SupervisorHealthCheck;
     import org.apache.storm.daemon.supervisor.timer.SupervisorHeartbeat;
     import org.apache.storm.daemon.supervisor.timer.UpdateBlobs;
    +import org.apache.storm.event.EventManager;
     import org.apache.storm.event.EventManagerImp;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.localizer.AsyncLocalizer;
    +import org.apache.storm.localizer.ILocalizer;
     import org.apache.storm.localizer.Localizer;
     import org.apache.storm.messaging.IContext;
     import org.apache.storm.metric.StormMetricsRegistry;
     import org.apache.storm.scheduler.ISupervisor;
     import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
     import org.apache.storm.utils.Utils;
     import org.apache.storm.utils.VersionInfo;
    +import org.apache.zookeeper.data.ACL;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -import java.io.File;
    -import java.io.InterruptedIOException;
    -import java.util.Collection;
    -import java.util.Map;
    -import java.util.Set;
    -import java.util.concurrent.Callable;
    -
    -public class Supervisor {
    +public class Supervisor implements DaemonCommon, AutoCloseable {
         private static final Logger LOG = LoggerFactory.getLogger(Supervisor.class);
    +    private final Map<String, Object> conf;
    +    private final IContext sharedContext;
    +    private volatile boolean active;
    +    private final ISupervisor iSupervisor;
    +    private final Utils.UptimeComputer upTime;
    +    private final String stormVersion;
    +    private final IStormClusterState stormClusterState;
    +    private final LocalState localState;
    +    private final String supervisorId;
    +    private final String assignmentId;
    +    private final String hostName;
    +    // used for reporting used ports when heartbeating
    +    private final AtomicReference<Map<Long, LocalAssignment>> currAssignment;
    +    private final StormTimer heartbeatTimer;
    +    private final StormTimer eventTimer;
    +    private final StormTimer blobUpdateTimer;
    +    private final Localizer localizer;
    +    private final ILocalizer asyncLocalizer;
    +    private EventManager eventManager;
    +    private ReadClusterState readState;
    +    
    +    private Supervisor(ISupervisor iSupervisor) throws IOException {
    +        this(Utils.readStormConfig(), null, iSupervisor);
    +    }
    +    
    +    public Supervisor(Map<String, Object> conf, IContext sharedContext, ISupervisor iSupervisor) throws IOException {
    +        this.conf = conf;
    +        this.iSupervisor = iSupervisor;
    +        this.active = true;
    +        this.upTime = Utils.makeUptimeComputer();
    +        this.stormVersion = VersionInfo.getVersion();
    +        this.sharedContext = sharedContext;
    +        
    +        iSupervisor.prepare(conf, ConfigUtils.supervisorIsupervisorDir(conf));
    +        
    +        List<ACL> acls = null;
    +        if (Utils.isZkAuthenticationConfiguredStormServer(conf)) {
    +            acls = SupervisorUtils.supervisorZkAcls();
    +        }
    +
    +        try {
    +            this.stormClusterState = ClusterUtils.mkStormClusterState(conf, acls, new ClusterStateContext(DaemonType.SUPERVISOR));
    +        } catch (Exception e) {
    +            LOG.error("supervisor can't create stormClusterState");
    +            throw Utils.wrapInRuntime(e);
    +        }
    +
    +        try {
    +            this.localState = ConfigUtils.supervisorState(conf);
    +            this.localizer = Utils.createLocalizer(conf, ConfigUtils.supervisorLocalDir(conf));
    +            this.asyncLocalizer = new AsyncLocalizer(conf, this.localizer);
    +        } catch (IOException e) {
    +            throw Utils.wrapInRuntime(e);
    +        }
    +        this.supervisorId = iSupervisor.getSupervisorId();
    +        this.assignmentId = iSupervisor.getAssignmentId();
    +
    +        try {
    +            this.hostName = Utils.hostname(conf);
    +        } catch (UnknownHostException e) {
    +            throw Utils.wrapInRuntime(e);
    +        }
    +
    +        this.currAssignment = new AtomicReference<Map<Long, LocalAssignment>>(new HashMap<Long,LocalAssignment>());
    +
    +        this.heartbeatTimer = new StormTimer(null, new DefaultUncaughtExceptionHandler());
    +
    +        this.eventTimer = new StormTimer(null, new DefaultUncaughtExceptionHandler());
    +
    +        this.blobUpdateTimer = new StormTimer("blob-update-timer", new DefaultUncaughtExceptionHandler());
    +    }
    +    
    +    public String getId() {
    +        return supervisorId;
    +    }
         
    -    private SyncProcessEvent localSyncProcess;
    +    IContext getSharedContext() {
    +        return sharedContext;
    +    }
    +
    +    public Map<String, Object> getConf() {
    +        return conf;
    +    }
    +
    +    public ISupervisor getiSupervisor() {
    +        return iSupervisor;
    +    }
    +
    +    public Utils.UptimeComputer getUpTime() {
    +        return upTime;
    +    }
    +
    +    public String getStormVersion() {
    +        return stormVersion;
    +    }
    +
    +    public IStormClusterState getStormClusterState() {
    +        return stormClusterState;
    +    }
     
    -    public void setLocalSyncProcess(SyncProcessEvent localSyncProcess) {
    -        this.localSyncProcess = localSyncProcess;
    +    LocalState getLocalState() {
    +        return localState;
         }
     
    +    public String getAssignmentId() {
    +        return assignmentId;
    +    }
    +
    +    public String getHostName() {
    +        return hostName;
    +    }
    +
    +    public AtomicReference<Map<Long, LocalAssignment>> getCurrAssignment() {
    +        return currAssignment;
    +    }
    +
    +    public Localizer getLocalizer() {
    +        return localizer;
    +    }
    +    
    +    ILocalizer getAsyncLocalizer() {
    +        return asyncLocalizer;
    +    }
    +    
    +    EventManager getEventManger() {
    +        return eventManager;
    +    }
    +    
         /**
    -     * in local state, supervisor stores who its current assignments are another thread launches events to restart any dead processes if necessary
    -     * 
    -     * @param conf
    -     * @param sharedContext
    -     * @param iSupervisor
    -     * @return
    -     * @throws Exception
    +     * Launch the supervisor
          */
    -    public SupervisorManager mkSupervisor(final Map conf, IContext sharedContext, ISupervisor iSupervisor) throws Exception {
    -        SupervisorManager supervisorManager = null;
    -        try {
    -            LOG.info("Starting Supervisor with conf {}", conf);
    -            iSupervisor.prepare(conf, ConfigUtils.supervisorIsupervisorDir(conf));
    -            String path = ConfigUtils.supervisorTmpDir(conf);
    -            FileUtils.cleanDirectory(new File(path));
    -
    -            final SupervisorData supervisorData = new SupervisorData(conf, sharedContext, iSupervisor);
    -            Localizer localizer = supervisorData.getLocalizer();
    -
    -            SupervisorHeartbeat hb = new SupervisorHeartbeat(conf, supervisorData);
    -            hb.run();
    -            // should synchronize supervisor so it doesn't launch anything after being down (optimization)
    -            Integer heartbeatFrequency = Utils.getInt(conf.get(Config.SUPERVISOR_HEARTBEAT_FREQUENCY_SECS));
    -            supervisorData.getHeartbeatTimer().scheduleRecurring(0, heartbeatFrequency, hb);
    -
    -            Set<String> downloadedStormIds = SupervisorUtils.readDownLoadedStormIds(conf);
    -            for (String stormId : downloadedStormIds) {
    -                SupervisorUtils.addBlobReferences(localizer, stormId, conf);
    -            }
    -            // do this after adding the references so we don't try to clean things being used
    -            localizer.startCleaner();
    +    public void launch() throws Exception {
    +        LOG.info("Starting Supervisor with conf {}", conf);
    +        String path = ConfigUtils.supervisorTmpDir(conf);
    +        FileUtils.cleanDirectory(new File(path));
     
    -            EventManagerImp syncSupEventManager = new EventManagerImp(false);
    -            EventManagerImp syncProcessManager = new EventManagerImp(false);
    -
    -            SyncProcessEvent syncProcessEvent = null;
    -            if (ConfigUtils.isLocalMode(conf)) {
    -                localSyncProcess.init(supervisorData);
    -                syncProcessEvent = localSyncProcess;
    -            } else {
    -                syncProcessEvent = new SyncProcessEvent(supervisorData);
    -            }
    +        Localizer localizer = getLocalizer();
     
    -            SyncSupervisorEvent syncSupervisorEvent = new SyncSupervisorEvent(supervisorData, syncProcessEvent, syncSupEventManager, syncProcessManager);
    -            UpdateBlobs updateBlobsThread = new UpdateBlobs(supervisorData);
    -            RunProfilerActions runProfilerActionThread = new RunProfilerActions(supervisorData);
    +        SupervisorHeartbeat hb = new SupervisorHeartbeat(conf, this);
    +        hb.run();
    +        // should synchronize supervisor so it doesn't launch anything after being down (optimization)
    +        Integer heartbeatFrequency = Utils.getInt(conf.get(Config.SUPERVISOR_HEARTBEAT_FREQUENCY_SECS));
    +        heartbeatTimer.scheduleRecurring(0, heartbeatFrequency, hb);
     
    -            if ((Boolean) conf.get(Config.SUPERVISOR_ENABLE)) {
    -                StormTimer eventTimer = supervisorData.getEventTimer();
    -                // This isn't strictly necessary, but it doesn't hurt and ensures that the machine stays up
    -                // to date even if callbacks don't all work exactly right
    -                eventTimer.scheduleRecurring(0, 10, new EventManagerPushCallback(syncSupervisorEvent, syncSupEventManager));
    +        this.eventManager = new EventManagerImp(false);
    +        this.readState = new ReadClusterState(this);
    +        
    +        Set<String> downloadedStormIds = SupervisorUtils.readDownLoadedStormIds(conf);
    --- End diff --
    
    Nitpick: Download isn't two words


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78230541
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,780 @@
    +/**
    + * 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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        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 + " }";
    --- End diff --
    
    nit, spaces around first +.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76475319
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,437 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = -1;
    +        _assignment = null;
    +        _workerId = workerId;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq"+pid);
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            if (in.readLine() != null) {
    +                ret = true;
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    private boolean isPosixProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("ps", "-o", "user", "-p", String.valueOf(pid));
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            String first = in.readLine();
    +            assert("USER".equals(first));
    +            String processUser;
    +            while ((processUser = in.readLine()) != null) {
    +                if (user.equals(processUser)) {
    +                    ret = true;
    +                    break;
    +                } else {
    +                    LOG.info("Found {} running as {}, but expected it to be {}", pid, processUser, user);
    +                }
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    @Override
    +    public boolean areAllProcessesDead() throws IOException {
    +        Set<Long> pids = getAllPids();
    +        String user = getWorkerUser();
    +        
    +        boolean allDead = true;
    +        for (Long pid: pids) {
    +            if (!isProcessAlive(pid, user)) {
    +                LOG.warn("{}: PID {} is dead", _workerId, pid);
    +            } else {
    +                allDead = false;
    +                break;
    +            }
    +        }
    +        return allDead;
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +    }
    +
    +    /**
    +     * Setup the container to run.  By default this creates the needed directories/links in the
    +     * local file system
    +     * PREREQUISITE: All needed blobs and topology, jars/configs have been downloaded and
    +     * placed in the appropriate locations
    +     * @throws IOException on any error
    +     */
    +    protected void setup() throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    +        }
    +        final String topologyId = _assignment.get_topology_id();
    +        if (!SupervisorUtils.doRequiredTopoFilesExist(_conf, topologyId)) {
    +            LOG.info("Missing topology storm code, so can't launch  worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                    _supervisorId, _port, _workerId);
    +            throw new IllegalStateException("Not all needed files are here!!!!");
    +        }
    +        String pidsPath = ConfigUtils.workerPidsRoot(_conf, _workerId);
    +        String hbPath = ConfigUtils.workerHeartbeatsRoot(_conf, _workerId);
    +    
    +        FileUtils.forceMkdir(new File(pidsPath));
    +        FileUtils.forceMkdir(new File(ConfigUtils.workerTmpRoot(_conf, _workerId)));
    +        FileUtils.forceMkdir(new File(hbPath));
    +    
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        String user = (String) topologyConf.get(Config.TOPOLOGY_SUBMITTER_USER);
    +        writeLogMetadata(topologyConf, user, topologyId);
    +        ConfigUtils.setWorkerUserWSE(_conf, _workerId, user);
    +        createArtifactsLink(topologyId);
    +    
    +        createBlobstoreLinks(topologyId);
    +    }
    +    
    +    /**
    +     * Write out the file used by the log viewer to allow/reject log access
    +     * @param topologyConf the config for the topology
    +     * @param user the user this is going to run as
    +     * @param topologyId the id of the topology
    +     * @throws IOException on any error
    +     */
    +    @SuppressWarnings("unchecked")
    +    protected void writeLogMetadata(Map<String, Object> topologyConf, String user, String topologyId) throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    --- End diff --
    
    cut & paste?


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @harshach I have +1 from enough people to merge this in and would really like to move forward with it, but if you are still looking at it I want to be sure you get a chance to finish.  Please let me know either way.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    I've left a comment of regression issue, but other manual tests passed including failed things.
    I'd be +1 once regression issue is resolved. Great works.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78942716
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,335 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.FileOutputStream;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.io.Writer;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    +    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
    +    
    +    /**
    +     * Factory to create a new AdvancedFSOps
    +     * @param conf the configuration of the process
    +     * @return the appropriate instance of the class for this config and environment.
    +     */
    +    public static AdvancedFSOps make(Map<String, Object> conf) {
    +        if (Utils.isOnWindows()) {
    +            return new AdvancedWindowsFSOps(conf);
    +        }
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new AdvancedRunAsUserFSOps(conf);
    +        }
    +        return new AdvancedFSOps();
    +    }
    +    
    +    private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
    +        private final Map<String, Object> _conf;
    +        
    +        public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
    +            if (Utils.isOnWindows()) {
    +                throw new UnsupportedOperationException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +            _conf = conf;
    +        }
    +        
    +        @Override
    +        public void setupBlobPermissions(File path, String user) throws IOException {
    +            String logPrefix = "setup blob permissions for " + path;
    +            SupervisorUtils.processLauncherAndWait(_conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
    +        }
    +        
    +        @Override
    +        public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +            String absolutePath = path.getAbsolutePath();
    +            LOG.info("Deleting path {}", absolutePath);
    +            if (user == null) {
    +                user = Files.getOwner(path.toPath()).getName();
    +            }
    +            List<String> commands = new ArrayList<>();
    +            commands.add("rmr");
    +            commands.add(absolutePath);
    +            SupervisorUtils.processLauncherAndWait(_conf, user, commands, null, logPrefix);
    +            if (Utils.checkFileExists(absolutePath)) {
    +                throw new RuntimeException(path + " was not deleted.");
    +            }
    +        }
    +        
    +        @Override
    +        public void setupStormCodeDir(Map<String, Object> topologyConf, File path) throws IOException {
    +            SupervisorUtils.setupStormCodeDir(_conf, topologyConf, path.getCanonicalPath());
    +        }
    +    }
    +    
    +    /**
    +     * Operations that need to override the default ones when running on Windows
    +     *
    +     */
    +    private static class AdvancedWindowsFSOps extends AdvancedFSOps {
    +
    +        public AdvancedWindowsFSOps(Map<String, Object> conf) {
    +            if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +        }
    +        
    +        @Override
    +        public void restrictDirectoryPermissions(File dir) throws IOException {
    +            //NOOP, if windows gets support for run as user we will need to find a way to suppor this
    +        }
    +        
    +        @Override
    +        public void moveDirectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // This is not atomic but it does work
    +            FileUtils.moveDirectory(fromDir, toDir);
    +        }
    +        
    +        @Override
    +        public boolean supportsAtomicDirectoryMove() {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // FileUtils.moveDirectory is not atomic
    +            return false;
    +        }
    +    }
    +    
    +    
    +    protected AdvancedFSOps() {
    +        //NOOP, but restricted permissions
    +    }
    +
    +    /**
    +     * Set directory permissions to (OWNER)RWX (GROUP)R-X (OTHER)---
    +     * On some systems that do not support this, it may become a noop
    +     * @param dir the directory to change permissions on
    +     * @throws IOException on any error
    +     */
    +    public void restrictDirectoryPermissions(File dir) throws IOException {
    +        Set<PosixFilePermission> perms = new HashSet<>(
    +                Arrays.asList(PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE,
    +                        PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ,
    +                        PosixFilePermission.GROUP_EXECUTE));
    +        Files.setPosixFilePermissions(dir.toPath(), perms);
    +    }
    +
    +    /**
    +     * Move fromDir to toDir, and try to make it an atomic move if possible
    +     * @param fromDir what to move
    +     * @param toDir where to move it from
    +     * @throws IOException on any error
    +     */
    +    public void moveDirectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +        FileUtils.forceMkdir(toDir);
    +        Files.move(fromDir.toPath(), toDir.toPath(), StandardCopyOption.ATOMIC_MOVE);
    +    }
    +    
    +    /**
    +     * @return true if an atomic directory move works, else false.
    +     */
    +    public boolean supportsAtomicDirectoryMove() {
    +        return true;
    +    }
    +    
    +    /**
    +     * Copy a directory
    +     * @param fromDir from where
    +     * @param toDir to where
    +     * @throws IOException on any error
    +     */
    +    public void copyDirectory(File fromDir, File toDir) throws IOException {
    +        FileUtils.copyDirectory(fromDir, toDir);
    +    }
    +    
    +    /**
    +     * Setup permissions properly for an internal blob store path
    +     * @param path the path to set the permissions on
    +     * @param user the user to change the permissions for
    +     * @throws IOException on any error
    +     */
    +    public void setupBlobPermissions(File path, String user) throws IOException {
    +        //Normally this is a NOOP
    +    }
    +
    +    /**
    +     * Delete a file or a directory and all of the children. If it exists.
    +     * @param path what to delete
    +     * @param user who to delete it as if doing it as someone else is supported
    +     * @param logPrefix if an external process needs to be launched to delete 
    +     * the object what prefix to include in the logs
    +     * @throws IOException on any error.
    +     */
    +    public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +        //by default no need to do this as a different user
    +        deleteIfExists(path);
    +    }
    +    
    +    /**
    +     * Delete a file or a directory and all of the children. If it exists.
    +     * @param path what to delete
    +     * @throws IOException on any error.
    +     */
    +    public void deleteIfExists(File path) throws IOException {
    +        LOG.info("Deleting path {}", path);
    +        Path p = path.toPath();
    +        if (Files.exists(p)) {
    +            try {
    +                FileUtils.forceDelete(path);
    +            } catch (FileNotFoundException ignored) {}
    +        }
    +    }
    +
    +    /**
    +     * Setup the permissions for the storm code dir
    +     * @param topologyConf the config of the Topology
    +     * @param path the directory to set the permissions on
    +     * @throws IOException on any error
    +     */
    +    public void setupStormCodeDir(Map<String, Object> topologyConf, File path) throws IOException {
    +        //By default this is a NOOP
    +    }
    +
    +    /**
    +     * Sanity check if everything the topology needs is there for it to run.
    +     * @param conf the config of the supervisor
    +     * @param topologyId the ID of the topology
    +     * @return true if everything is there, else false
    +     * @throws IOException on any error
    +     */
    +    public boolean doRequiredTopoFilesExist(Map<String, Object> conf, String topologyId) throws IOException {
    +        return SupervisorUtils.doRequiredTopoFilesExist(conf, topologyId);
    +    }
    +    
    +    /**
    +     * Makes a directory, including any necessary but nonexistent parent
    +     * directories. 
    +     *
    +     * @param path the directory to create
    +     * @throws IOException on any error
    +     */
    +    public void forceMkdir(File path) throws IOException {
    +        FileUtils.forceMkdir(path);
    +    }
    +    
    +    /**
    +     * Check if a file exists or not
    +     * @param path the path the check
    --- End diff --
    
    Nitpick: Should be "the path to check"


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76460431
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -211,4 +209,15 @@ public void releaseResourcesForWorker(String workerId) {
         public void close() throws IOException {
             this.center.deleteCgroup(this.rootCgroup);
         }
    +
    +    @Override
    +    public Set<Long> getRunningPIDs(String workerId) throws IOException {
    +        CgroupCommon workerGroup = new CgroupCommon(workerId, this.hierarchy, this.rootCgroup);
    +        if (!this.rootCgroup.getChildren().contains(workerGroup)) {
    +            LOG.warn("cgroup {} doesn't exist!", workerGroup);
    +            return null;
    +        }
    +        Set<Long> ret = new HashSet<>();
    --- End diff --
    
    `ret` isn't used here, but we could return an empty set instead of null.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77369914
  
    --- 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);
    +            }
    +            if (container != null) {
    +                sb.append(" container: ");
    +                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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WAITING_FOR_WORKER_START) {
    +                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;
    +                    } else if (aHasResources && bHasResources) {
    --- End diff --
    
    Likewise this can be an `if` on its own line.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77361882
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/RunAsUserContainer.java ---
    @@ -0,0 +1,107 @@
    +/**
    + * 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.File;
    +import java.io.IOException;
    +import java.util.Arrays;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class RunAsUserContainer extends BasicContainer {
    +    private static final Logger LOG = LoggerFactory.getLogger(RunAsUserContainer.class);
    +
    +    public RunAsUserContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState state, ResourceIsolationInterface resourceIsolationManager, boolean recover) throws IOException {
    +        super(port, assignment, conf, supervisorId, state, resourceIsolationManager, recover);
    +        if (Utils.isOnWindows()) {
    +            throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +        }
    +    }
    +
    +    public RunAsUserContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        super(workerId, conf, supervisorId, resourceIsolationManager);
    +        if (Utils.isOnWindows()) {
    +            throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    --- End diff --
    
    UnsupportedOperationException, and in fact we could add a Utils#unsupportedOnWindows method and just call it as we would a validator.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77421346
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    --- End diff --
    
    OK.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77321291
  
    --- Diff: storm-core/src/jvm/org/apache/storm/utils/Utils.java ---
    @@ -1990,31 +1995,26 @@ protected void forceDeleteImpl(String path) throws IOException {
         }
     
         /**
    -     * Creates a symbolic link to the target
    +     * Creates a symbolic link to the target and force the creation if the target already exists
          * @param dir the parent directory of the link
          * @param targetDir the parent directory of the link's target
    -     * @param filename the file name of the link
          * @param targetFilename the file name of the links target
    +     * @param filename the file name of the link
          * @throws IOException
          */
    -    public static void createSymlink(String dir, String targetDir,
    -            String filename, String targetFilename) throws IOException {
    +    public static void forceCreateSymlink(String dir, String targetDir,
    --- End diff --
    
    The sequence of parameters have been swapped: targetFilename and filename.
    Since parameters of parent directory are link and target, it would be natural to keep that sequence to filename, too.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76609228
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,569 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState localState, ResourceIsolationInterface resourceIsolationManager, boolean recover)
    +            throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException(
    +                    "Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            _localState.setApprovedWorkers(workersToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = null;
    +        try (FileReader reader = new FileReader(str); BufferedReader br = new BufferedReader(reader)) {
    +            workerPid = br.readLine().trim();
    +        }
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        return runProfilingCommand(command, env, logPrefix, targetFile);
    +    }
    +
    +    /**
    +     * Get the command to run when doing profiling
    +     * @param action the profiling action to perform
    +     * @param stop if this is meant to stop the profiling or start it
    +     * @param workerPid the PID of the process to profile
    +     * @param targetDir the current working directory of the worker process
    +     * @return the command to run for profiling.
    +     */
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        if (action == ProfileAction.JMAP_DUMP) {
    --- End diff --
    
    I copied and pasted this, but I agree a switch would be cleaner.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77532348
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,769 @@
    +/**
    + * 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);
    +            }
    +            if (container != null) {
    +                sb.append(" container: ");
    +                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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WAITING_FOR_WORKER_START) {
    +                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 equivilant(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;
    +                    } else 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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), 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.get_topology_id(), 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);
    +        
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivilant(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +        
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for WAITING_FOR_WORKER_START state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleWaitingForWorkerStart(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb != null) {
    +            long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +            if (hbAgeMs <= staticState.hbTimeoutMs) {
    +                return dynamicState.withState(MachineState.RUNNING);
    +            }
    +        }
    +        
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            //We were rescheduled while waiting for the worker to come up
    +            return Slot.killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffms = (Time.currentTimeMillis() - dynamicState.startTime);
    +        if (timeDiffms > staticState.firstHbTimeoutMs) {
    +            LOG.warn("SLOT {}: Container {} failed to launch in {} ms.", staticState.port, dynamicState.container, staticState.firstHbTimeoutMs);
    +            dynamicState.container.kill();
    +            Time.sleep(staticState.killSleepMs);
    +            return dynamicState.withState(MachineState.KILL_AND_RELAUNCH);
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for RUNNING state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleRunning(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            LOG.warn("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAssignment, dynamicState.newAssignment);
    +            //Scheduling changed while running...
    +            return killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        if (dynamicState.container.didMainProcessExit()) {
    +            LOG.warn("SLOT {}: main process has exited", staticState.port);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb == null) {
    +            LOG.warn("SLOT {}: HB returned as null", staticState.port);
    +            //This should never happen, but to be safe
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +        if (timeDiffMs > staticState.hbTimeoutMs) {
    +            LOG.warn("SLOT {}: HB is too old {} > {}", staticState.port, timeDiffMs, staticState.hbTimeoutMs);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        //The worker is up and running check for profiling requests
    +        if (!dynamicState.profileActions.isEmpty()) {
    +            HashSet<TopoProfileAction> mod = new HashSet<>(dynamicState.profileActions);
    +            HashSet<TopoProfileAction> modPending = new HashSet<>(dynamicState.pendingStopProfileActions);
    +            Iterator<TopoProfileAction> iter = mod.iterator();
    +            while (iter.hasNext()) {
    +                TopoProfileAction action = iter.next();
    +                if (!action.topoId.equals(dynamicState.currentAssignment.get_topology_id())) {
    +                    iter.remove();
    +                    LOG.warn("Dropping {} wrong topology is running", action);
    +                    //Not for this topology so skip it
    +                } else {
    +                    if (modPending.contains(action)) {
    +                        boolean isTimeForStop = Time.currentTimeMillis() > action.request.get_time_stamp();
    +                        if (isTimeForStop) {
    +                            if (dynamicState.container.runProfiling(action.request, true)) {
    +                                LOG.debug("Stopped {} action finished", action);
    +                                iter.remove();
    +                                modPending.remove(action);
    +                            } else {
    +                                LOG.warn("Stopping {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            LOG.debug("Still pending {} now: {}", action, Time.currentTimeMillis());
    +                        }
    +                    } else {
    +                        //J_PROFILE_START is not used.  When you see a J_PROFILE_STOP
    +                        // start profiling and save it away to stop when timeout happens
    +                        if (action.request.get_action() == ProfileAction.JPROFILE_STOP) {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                modPending.add(action);
    +                                LOG.debug("Started {} now: {}", action, Time.currentTimeMillis());
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                LOG.debug("Started {} action finished", action);
    +                                iter.remove();
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        }
    +                    }
    +                }
    +            }
    +            dynamicState = dynamicState.withProfileActions(mod, modPending);
    +        }
    +        Time.sleep(staticState.monitorFreqMs);
    +        return dynamicState;
    +    }
    +
    +    static DynamicState handleEmpty(DynamicState dynamicState, StaticState staticState) throws InterruptedException {
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +        }
    +        //Both assignments are null, just wait
    +        if (dynamicState.profileActions != null && !dynamicState.profileActions.isEmpty()) {
    +            //Nothing is scheduled here so throw away all of the profileActions
    +            LOG.warn("Dropping {} no topology is running", dynamicState.profileActions);
    +            dynamicState = dynamicState.withProfileActions(Collections.<TopoProfileAction> emptySet(), Collections.<TopoProfileAction> emptySet());
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +    
    +    private final AtomicReference<LocalAssignment> newAssignment = new AtomicReference<>();
    +    private final AtomicReference<Set<TopoProfileAction>> profiling =
    +            new AtomicReference<Set<TopoProfileAction>>(new HashSet<TopoProfileAction>());
    +    private final StaticState staticState;
    +    private final IStormClusterState clusterState;
    +    private volatile boolean done = false;
    +    private volatile DynamicState dynamicState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignmants;
    +    
    +    public Slot(ILocalizer localizer, Map<String, Object> conf, 
    +            ContainerLauncher containerLauncher, String host,
    +            int port, LocalState localState,
    +            IStormClusterState clusterState,
    +            ISupervisor iSupervisor,
    +            AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignmants) throws Exception {
    +        super("SLOT_"+port);
    +
    +        this.cachedCurrentAssignmants = cachedCurrentAssignmants;
    +        this.clusterState = clusterState;
    +        Map<Integer, LocalAssignment> assignments = localState.getLocalAssignmentsMap();
    +        LocalAssignment currentAssignment = null;
    +        if (assignments != null) {
    +            currentAssignment = assignments.get(port);
    +        }
    +        Container container = null;
    +        if (currentAssignment != null) { 
    +            container = containerLauncher.recoverContainer(port, currentAssignment, localState);
    +        }
    +        
    +        LocalAssignment newAssignment = currentAssignment;
    +        if (currentAssignment != null && container == null) {
    +            currentAssignment = null;
    +            //Assigned something but it is not running
    +        }
    +        
    +        dynamicState = new DynamicState(currentAssignment, container, newAssignment);
    +        staticState = new StaticState(localizer, 
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_START_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_MONITOR_FREQUENCY_SECS)) * 1000,
    +                containerLauncher,
    +                host,
    +                port,
    +                iSupervisor,
    +                localState);
    +        this.newAssignment.set(dynamicState.newAssignment);
    +        if (MachineState.RUNNING == dynamicState.state) {
    +            //We are running so we should recover the blobs.
    +            staticState.localizer.recoverRunningTopology(currentAssignment.get_topology_id(), port);
    +        }
    +        LOG.warn("SLOT {}:{} Starting in state {} - assignment {}", staticState.host, staticState.port, dynamicState.state, dynamicState.currentAssignment);
    +    }
    +    
    +    public MachineState getMachineState() {
    +        return dynamicState.state;
    +    }
    +    
    +    /**
    +     * Set a new assignment asynchronously
    +     * @param newAssignment the new assignment for this slot to run, null to run nothing
    +     */
    +    public void setNewAssignment(LocalAssignment newAssignment) {
    +        this.newAssignment.set(newAssignment);
    +    }
    +    
    +    public void addProfilerActions(Set<TopoProfileAction> actions) {
    +        if (actions != null) {
    +            while(true) {
    +                Set<TopoProfileAction> orig = profiling.get();
    +                Set<TopoProfileAction> newActions = new HashSet<>(orig);
    +                newActions.addAll(actions);
    +                if (profiling.compareAndSet(orig, newActions)) {
    +                    return;
    +                }
    +            }
    +        }
    +    }
    +    
    +    public void run() {
    +        try {
    +            while(!done) {
    +                LocalAssignment localNewAssignment = newAssignment.get();
    +                Set<TopoProfileAction> origProfileActions = new HashSet<>(profiling.get());
    +                Set<TopoProfileAction> removed = new HashSet<>(origProfileActions);
    +                
    +                DynamicState nextState = 
    +                        stateMachineStep(dynamicState.withNewAssignment(localNewAssignment)
    +                                .withProfileActions(origProfileActions, dynamicState.pendingStopProfileActions), staticState);
    +
    +                if (LOG.isDebugEnabled() || dynamicState.state != nextState.state) {
    +                    LOG.info("STATE {} -> {}", dynamicState, nextState);
    +                }
    +                //Save the current state for recovery
    +                if (!equivilant(nextState.currentAssignment, dynamicState.currentAssignment)) {
    +                    LOG.warn("SLOT {}: Changing current assignment from {} to {}", staticState.port, dynamicState.currentAssignment, nextState.currentAssignment);
    +                    synchronized(staticState.localState) {
    +                        Map<Integer, LocalAssignment> assignments = staticState.localState.getLocalAssignmentsMap();
    +                        if (assignments == null) {
    +                            assignments = new HashMap<>();
    +                        }
    +                        if (nextState.currentAssignment == null) {
    +                            assignments.remove(staticState.port);
    +                        } else {
    +                            assignments.put(staticState.port, nextState.currentAssignment);
    +                        }
    +                        staticState.localState.setLocalAssignmentsMap(assignments);
    +                    }
    +                    Map<Long, LocalAssignment> update = null;
    +                    Map<Long, LocalAssignment> orig = null;
    +                    do {
    +                        Long lport = new Long(staticState.port);
    +                        orig = cachedCurrentAssignmants.get();
    +                        update = new HashMap<>(orig);
    +                        if (nextState.currentAssignment == null) {
    +                            update.remove(lport);
    +                        } else {
    +                            update.put(lport, nextState.currentAssignment);
    +                        }
    +                    } while (!cachedCurrentAssignmants.compareAndSet(orig, update));
    +                }
    +                
    +                // clean up the profiler actions that are not being processed
    +                removed.removeAll(dynamicState.profileActions);
    +                removed.removeAll(dynamicState.pendingStopProfileActions);
    +                for (TopoProfileAction action: removed) {
    +                    try {
    +                        clusterState.deleteTopologyProfileRequests(action.topoId, action.request);
    +                    } catch (Exception e) {
    +                        LOG.error("Error trying to remove profiling request, it will be retried", e);
    +                    }
    +                }
    +                Set<TopoProfileAction> orig, copy;
    +                do {
    +                    orig = profiling.get();
    +                    copy = new HashSet<>(orig);
    +                    copy.removeAll(removed);
    +                } while (!profiling.compareAndSet(orig, copy));
    +                dynamicState = nextState;
    +                newAssignment.compareAndSet(localNewAssignment, dynamicState.newAssignment);
    --- End diff --
    
    Ok. Thanks! Took me a while to go through Slot and the state machine. The unit tests were very helpful on that.
    
    To summarize how I understand this:
    1. we get a copy of new assignment
    2. mutate localstate assignments
    3. mutate supervisor in-memory assingments
    4. deal with profiling cleanup
    5. persist the new assignment in slot.
    6. If new assignment persisted successfully, then we are good. The equivalency checks will pass, and the state machine should progress or stay at RUNNING. Else, the equivalency checks will move us to correct state. Also this loop fixes up localstate and in-memory assignments continuously.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77418422
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,420 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + * TODO once we have replaced the original supervisor merge this with
    + * Localizer and optimize them
    + */
    +public class AsyncLocalizer implements ILocalizer, Shutdownable {
    +    /**
    +     * A future that has already completed.
    +     */
    +    private static class AllDoneFuture implements Future<Void> {
    +
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isCancelled() {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isDone() {
    +            return true;
    +        }
    +
    +        @Override
    +        public Void get() {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void get(long timeout, TimeUnit unit) {
    +            return null;
    +        }
    +
    +    }
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizer.class);
    +
    +    private final Localizer _localizer;
    +    private final ExecutorService _execService;
    +    private final boolean _isLocalMode;
    +    private final Map<String, Object> _conf;
    +    private final Map<String, LocalDownloadedResource> _basicPending;
    +    private final Map<String, LocalDownloadedResource> _blobPending;
    +    private final AdvancedFSOps _fsOps;
    +
    +    private class DownloadBaseBlobsDistributed implements Callable<Void> {
    +        private final String _topologyId;
    +        
    +        public DownloadBaseBlobsDistributed(String topologyId) {
    +            this._topologyId = topologyId;
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            File sr = new File(stormroot);
    +            if (sr.exists()) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    Utils.forceDelete(stormroot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            try {
    +                String stormJarKey = ConfigUtils.masterStormJarKey(_topologyId);
    +                String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +                String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +                String jarPath = ConfigUtils.supervisorStormJarPath(tmproot);
    +                String codePath = ConfigUtils.supervisorStormCodePath(tmproot);
    +                String confPath = ConfigUtils.supervisorStormConfPath(tmproot);
    +                FileUtils.forceMkdir(new File(tmproot));
    +                _fsOps.restrictDirectoryPermissions(tmproot);
    +                ClientBlobStore blobStore = Utils.getClientBlobStoreForSupervisor(_conf);
    +                try {
    +                    Utils.downloadResourcesAsSupervisor(stormJarKey, jarPath, blobStore);
    +                    Utils.downloadResourcesAsSupervisor(stormCodeKey, codePath, blobStore);
    +                    Utils.downloadResourcesAsSupervisor(stormConfKey, confPath, blobStore);
    +                } finally {
    +                    blobStore.shutdown();
    +                }
    +                Utils.extractDirFromJar(jarPath, ConfigUtils.RESOURCES_SUBDIR, tmproot);
    +                _fsOps.moveDirectoryPreferAtomic(new File(tmproot), new File(stormroot));
    +                SupervisorUtils.setupStormCodeDir(_conf, ConfigUtils.readSupervisorStormConf(_conf, _topologyId), stormroot);
    +                deleteAll = false;
    +            } finally {
    +                if (deleteAll) {
    +                    LOG.info("Failed to download basic resources for topology-id {}", _topologyId);
    +                    Utils.forceDelete(tmproot);
    +                    Utils.forceDelete(stormroot);
    --- End diff --
    
    We can catch and ignore any IOException from these forceDelete calls so that the original exception bubbles up.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76873417
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,493 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.io.Reader;
    +import java.io.Writer;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected final Map<String, Object> _topoConf;
    +    protected String _workerId;
    +    protected final String _topologyId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    //Exposed for testing
    +    protected Container(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        assert((assignment == null && port <= 0) ||
    +                (assignment != null && port > 0));
    +        assert(conf != null);
    +        assert(ops != null);
    +        assert(supervisorId != null);
    +        
    +        _port = port;
    +        _ops = ops;
    +        _assignment = assignment;
    +        if (assignment != null) {
    +            _topologyId = assignment.get_topology_id();
    +        } else {
    +            _topologyId = null;
    +        }
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        if (topoConf == null) {
    +            _topoConf = readTopoConf();
    +        } else {
    +            _topoConf = topoConf;
    +        }
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return this.getClass().getSimpleName() + " topo:" + _topologyId + " worker:" + _workerId;
    +    }
    +    
    +    protected Map<String, Object> readTopoConf() throws IOException {
    +        assert(_topologyId != null);
    +        return ConfigUtils.readSupervisorStormConf(_conf, _topologyId);
    +    }
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), port, assignment, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     * @throws IOException on any error
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), -1, null, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.trace("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    --- End diff --
    
    I would like to but I don't really have access to a windows box so I didn't feel comfortable parsing the output of the windows function without testing it.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77365678
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/RunAsUserContainerLauncher.java ---
    @@ -0,0 +1,71 @@
    +/**
    + * 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.Map;
    +
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.utils.LocalState;
    +
    +public class RunAsUserContainerLauncher extends ContainerLauncher {
    +    private final Map<String, Object> _conf;
    +    private final String _supervisorId;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    public RunAsUserContainerLauncher(Map<String, Object> conf, String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +    }
    +
    +    @Override
    +    public Container launchContainer(int port, LocalAssignment assignment, LocalState state) throws IOException {
    +        Container container = new RunAsUserContainer(port, assignment, _conf, _supervisorId, state,
    +                _resourceIsolationManager, false);
    +        container.setup();
    +        container.launch();
    +        return container;
    +    }
    +
    +    @Override
    +    public Container recoverContainer(int port, LocalAssignment assignment, LocalState state) throws IOException {
    +        Container container = null;
    +        try {
    +            container = new RunAsUserContainer(port, assignment, _conf, _supervisorId, state, 
    +                    _resourceIsolationManager, true);
    +        } catch (ContainerRecoveryException e) {
    +            // We could not recover return null
    +        }
    +        return container;
    +    }
    +    
    +    @Override
    +    public Killable recoverContainer(String workerId) throws IOException {
    +        Container container = null;
    +        try {
    +            container = new RunAsUserContainer(workerId, _conf, _supervisorId, 
    +                    _resourceIsolationManager);
    +        } catch (ContainerRecoveryException e) {
    +            // We could not recover return null
    +        }
    +        return container;
    +    }
    --- End diff --
    
    Could just `return RunAsUserContainer(w, c, s, r);`


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77527810
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,420 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + * TODO once we have replaced the original supervisor merge this with
    + * Localizer and optimize them
    + */
    +public class AsyncLocalizer implements ILocalizer, Shutdownable {
    +    /**
    +     * A future that has already completed.
    +     */
    +    private static class AllDoneFuture implements Future<Void> {
    +
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isCancelled() {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isDone() {
    +            return true;
    +        }
    +
    +        @Override
    +        public Void get() {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void get(long timeout, TimeUnit unit) {
    --- End diff --
    
    The only `@NotNull` I could find that java could be using is a sun internal annotation.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77457645
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,768 @@
    +/**
    + * 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);
    +            if (state == MachineState.WAITING_FOR_WORKER_START ||
    +                state == MachineState.KILL ||
    +                state == MachineState.KILL_AND_RELAUNCH) {
    +                sb.append(" msInState: ");
    +                sb.append(Time.currentTimeMillis() - startTime);
    +            }
    +            if (container != null) {
    +                sb.append(" container: ");
    +                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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WAITING_FOR_WORKER_START) {
    +                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;
    +                    } else 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.get_topology_id(), 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.get_topology_id(), 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.get_topology_id(), 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);
    +        
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        //The child processes typically exit in < 1 sec.  If 2 mins later they are still around something is wrong
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for WAITING_FOR_WORKER_START state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleWaitingForWorkerStart(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb != null) {
    +            long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +            if (hbAgeMs <= staticState.hbTimeoutMs) {
    +                return dynamicState.withState(MachineState.RUNNING);
    +            }
    +        }
    +        
    +        if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            //We were rescheduled while waiting for the worker to come up
    +            return Slot.killContainerForChangedAssignment(dynamicState, staticState);
    --- End diff --
    
    Add log line to make it consistent with other calls to killContainerForChangedAssignment. Also no need for Slot.
    
    LOG.warn("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAs    signment, dynamicState.newAssignment);


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78972625
  
    --- 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);
    +        
    +        //Ignore changes to scheduling while downloading the topology blobs
    +        // We don't support canceling the download through the future yet,
    +        // so to keep everything in sync, just wait
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization, staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    --- End diff --
    
    Consider asserting state is empty


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76309477
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Supervisor.java ---
    @@ -17,135 +17,541 @@
      */
     package org.apache.storm.daemon.supervisor;
     
    +import java.io.File;
    +import java.io.IOException;
    +import java.net.UnknownHostException;
    +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.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
     import org.apache.commons.io.FileUtils;
     import org.apache.storm.Config;
     import org.apache.storm.StormTimer;
    -import org.apache.storm.daemon.supervisor.timer.RunProfilerActions;
    +import org.apache.storm.cluster.ClusterStateContext;
    +import org.apache.storm.cluster.ClusterUtils;
    +import org.apache.storm.cluster.DaemonType;
    +import org.apache.storm.cluster.IStateStorage;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.daemon.DaemonCommon;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
     import org.apache.storm.daemon.supervisor.timer.SupervisorHealthCheck;
     import org.apache.storm.daemon.supervisor.timer.SupervisorHeartbeat;
     import org.apache.storm.daemon.supervisor.timer.UpdateBlobs;
    +import org.apache.storm.event.EventManager;
     import org.apache.storm.event.EventManagerImp;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.AsyncLocalizer;
    +import org.apache.storm.localizer.ILocalizer;
     import org.apache.storm.localizer.Localizer;
     import org.apache.storm.messaging.IContext;
     import org.apache.storm.metric.StormMetricsRegistry;
     import org.apache.storm.scheduler.ISupervisor;
     import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
     import org.apache.storm.utils.Utils;
     import org.apache.storm.utils.VersionInfo;
    +import org.apache.zookeeper.data.ACL;
    +import org.eclipse.jetty.util.ConcurrentHashSet;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -import java.io.File;
    -import java.io.InterruptedIOException;
    -import java.util.Collection;
    -import java.util.Map;
    -import java.util.Set;
    -import java.util.concurrent.Callable;
    +public class Supervisor implements DaemonCommon, AutoCloseable {
    +    
    +    public class ReadStateThread implements Runnable, AutoCloseable {
    +        private final Map<String, Object> superConf;
    +        private final IStormClusterState stormClusterState;
    +        private final EventManager syncSupEventManager;
    +        private final AtomicReference<Map<String, Map<String, Object>>> assignmentVersions;
    +        private final Map<Integer, Slot> slots = new HashMap<>();
    +        private final AtomicInteger readRetry = new AtomicInteger(0);
    +        private final String assignmentId;
    +        private final ISupervisor iSuper;
    +        private final ILocalizer localizer;
    +        private final ContainerLauncher launcher;
    +        private final String host;
    +        private final LocalState localState;
    +        private final IStormClusterState clusterState;
    +        private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +        
    +        public ReadStateThread(EventManager syncSupEventManager) throws Exception {
    +            this(getConf(), getStormClusterState(), syncSupEventManager,
    +                    getAssignmentId(), getiSupervisor(),
    +                    getAsyncLocalizer(), getHostName(),
    +                    getLocalState(), getStormClusterState(),
    +                    getCurrAssignment(), getSharedContext());
    +        }
    +        
    +        public ReadStateThread(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +                EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +                ILocalizer localizer, String host, LocalState localState,
    +                IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +                IContext sharedContext) throws Exception{
    +            this.superConf = superConf;
    +            this.stormClusterState = stormClusterState;
    +            this.syncSupEventManager = syncSupEventManager;
    +            this.assignmentVersions = new AtomicReference<Map<String, Map<String, Object>>>(new HashMap<String, Map<String, Object>>());
    +            this.assignmentId = assignmentId;
    +            this.iSuper = iSuper;
    +            this.localizer = localizer;
    +            this.host = host;
    +            this.localState = localState;
    +            this.clusterState = clusterState;
    +            this.cachedAssignments = cachedAssignments;
    +            
    +            this.launcher = ContainerLauncher.mk(superConf, assignmentId, sharedContext);
    +            
    +            @SuppressWarnings("unchecked")
    +            List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +            for (Number port: ports) {
    +                slots.put(port.intValue(), mkSlot(port.intValue()));
    +            }
    +        }
    +
    +        private Slot mkSlot(int port) throws Exception {
    +            Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                    localState, clusterState, iSuper, cachedAssignments);
    +            slot.start();
    +            return slot;
    +        }
    +        
    +        @Override
    +        public synchronized void run() {
    +            try {
    +                Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +                List<String> stormIds = stormClusterState.assignments(syncCallback);
    +                Map<String, Map<String, Object>> assignmentsSnapshot =
    +                        getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +                
    +                Map<Integer, LocalAssignment> allAssignments =
    +                        readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +                if (allAssignments == null) {
    +                    //Something odd happened try again later
    +                    return;
    +                }
    +                Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +                
    +                HashSet<Integer> assignedPorts = new HashSet<>();
    +                LOG.debug("Synchronizing supervisor");
    +                LOG.debug("All assignment: {}", allAssignments);
    +                LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +                for (Integer port: allAssignments.keySet()) {
    +                    if (iSuper.confirmAssigned(port)) {
    +                        assignedPorts.add(port);
    +                    }
    +                }
    +                HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +                allPorts.addAll(slots.keySet());
    +                
    +                Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +                for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                    String topoId = entry.getKey();
    +                    if (entry.getValue() != null) {
    +                        for (ProfileRequest req: entry.getValue()) {
    +                            NodeInfo ni = req.get_nodeInfo();
    +                            if (host.equals(ni.get_node())) {
    +                                Long port = ni.get_port().iterator().next();
    +                                Set<TopoProfileAction> actions = filtered.get(port);
    +                                if (actions == null) {
    +                                    actions = new HashSet<>();
    +                                    filtered.put(port.intValue(), actions);
    +                                }
    +                                actions.add(new TopoProfileAction(topoId, req));
    +                            }
    +                        }
    +                    }
    +                }
    +                
    +                for (Integer port: allPorts) {
    +                    Slot slot = slots.get(port);
    +                    if (slot == null) {
    +                        slot = mkSlot(port);
    +                        slots.put(port, slot);
    +                    }
    +                    slot.setNewAssignment(allAssignments.get(port));
    +                    slot.addProfilerActions(filtered.get(port));
    +                }
    +                
    +            } catch (Exception e) {
    +                LOG.error("Failed to Sync Supervisor", e);
    +                throw new RuntimeException(e);
    +            }
    +        }
    +        
    +        protected Map<String, Map<String, Object>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +                Map<String, Map<String, Object>> localAssignmentVersion, Runnable callback) throws Exception {
    +            Map<String, Map<String, Object>> updateAssignmentVersion = new HashMap<>();
    +            for (String topoId : topoIds) {
    +                Integer recordedVersion = -1;
    +                Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +                if (localAssignmentVersion.containsKey(topoId) && localAssignmentVersion.get(topoId) != null) {
    +                    recordedVersion = (Integer) localAssignmentVersion.get(topoId).get(IStateStorage.VERSION);
    +                }
    +                if (version == null) {
    +                    // ignore
    +                } else if (version == recordedVersion) {
    +                    updateAssignmentVersion.put(topoId, localAssignmentVersion.get(topoId));
    +                } else {
    +                    //TODO change this so we return an object not a map with magic keys
    +                    Map<String, Object> assignmentVersion = (Map<String, Object>) stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                    updateAssignmentVersion.put(topoId, assignmentVersion);
    +                }
    +            }
    +            return updateAssignmentVersion;
    +        }
    +        
    +        protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +            Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +            for (String stormId : stormIds) {
    +                List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +                ret.put(stormId, profileRequests);
    +            }
    +            return ret;
    +        }
    +        
    +        protected Map<Integer, LocalAssignment> readAssignments(Map<String, Map<String, Object>> assignmentsSnapshot,
    +                String assignmentId, AtomicInteger retries) {
    +            try {
    +                Map<Integer, LocalAssignment> portLA = new HashMap<Integer, LocalAssignment>();
    +                for (Map.Entry<String, Map<String, Object>> assignEntry : assignmentsSnapshot.entrySet()) {
    +                    String topoId = assignEntry.getKey();
    +                    Assignment assignment = (Assignment) assignEntry.getValue().get(IStateStorage.DATA);
    +
    +                    Map<Integer, LocalAssignment> portTasks = readMyExecutors(topoId, assignmentId, assignment);
    +
    +                    for (Map.Entry<Integer, LocalAssignment> entry : portTasks.entrySet()) {
    +
    +                        Integer port = entry.getKey();
    +
    +                        LocalAssignment la = entry.getValue();
    +
    +                        if (!portLA.containsKey(port)) {
    +                            portLA.put(port, la);
    +                        } else {
    +                            throw new RuntimeException("Should not have multiple topologys assigned to one port");
    +                        }
    +                    }
    +                }
    +                retries.set(0);
    +                return portLA;
    +            } catch (RuntimeException e) {
    +                if (retries.get() > 2) {
    --- End diff --
    
    This was hard coded before and I just moved the code over.  I also don't see a lot of value in making it configurable at this point.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76873184
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,629 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +
    +    //For testing purposes
    +    public BasicContainer(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager, LocalState localState,
    +            String profileCmd) throws IOException {
    +        super(ops, port, assignment, conf, topoConf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +        _profileCmd = profileCmd;
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState localState, ResourceIsolationInterface resourceIsolationManager, boolean recover)
    +            throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException(
    +                    "Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            _localState.setApprovedWorkers(workersToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        setup();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = _ops.slurpString(new File(str)).trim();
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        if (command.size() > 0) {
    +            return runProfilingCommand(command, env, logPrefix, targetFile);
    +        }
    +        LOG.warn("PROFILING REQUEST NOT SUPPORTED {} IGNORED...", request);
    +        return true;
    +    }
    +
    +    /**
    +     * Get the command to run when doing profiling
    +     * @param action the profiling action to perform
    +     * @param stop if this is meant to stop the profiling or start it
    +     * @param workerPid the PID of the process to profile
    +     * @param targetDir the current working directory of the worker process
    +     * @return the command to run for profiling.
    +     */
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        switch(action) {
    +            case JMAP_DUMP:
    +                return jmapDumpCmd(workerPid, targetDir);
    +            case JSTACK_DUMP:
    +                return jstackDumpCmd(workerPid, targetDir);
    +            case JPROFILE_DUMP:
    +                return jprofileDump(workerPid, targetDir);
    +            case JVM_RESTART:
    +                return jprofileJvmRestart(workerPid);
    +            case JPROFILE_STOP:
    +                if (stop) {
    +                    return jprofileStop(workerPid, targetDir);
    +                }
    +                return jprofileStart(workerPid);
    +            default:
    +                return Lists.newArrayList();
    +        }
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +
    +    /**
    +     * Compute the java.library.path that should be used for the worker.
    +     * This helps it to load JNI libraries that are packaged in the uber jar.
    +     * @param stormRoot the root directory of the worker process
    +     * @param conf the config for the supervisor.
    +     * @return the java.library.path/LD_LIBRARY_PATH to use so native libraries load correctly.
    +     */
    +    protected String jlp(String stormRoot, Map<String, Object> conf) {
    +        String resourceRoot = stormRoot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +        String os = System.getProperty("os.name").replaceAll("\\s+", "_");
    +        String arch = System.getProperty("os.arch");
    +        String archResourceRoot = resourceRoot + Utils.FILE_PATH_SEPARATOR + os + "-" + arch;
    +        String ret = CPJ.join(archResourceRoot, resourceRoot,
    +                conf.get(Config.JAVA_LIBRARY_PATH));
    +        return ret;
    +    }
    +
    +    /**
    +     * Returns a collection of jar file names found under the given directory.
    +     * @param dir the directory to search
    +     * @return the jar file names
    +     */
    +    protected List<String> getFullJars(File dir) {
    +        File[] files = dir.listFiles(jarFilter);
    +
    +        if (files == null) {
    +            return Collections.emptyList();
    +        }
    +
    +        return Arrays.stream(files).map(f -> f.getAbsolutePath())
    +                .collect(Collectors.toList());
    +    }
    +    
    +    protected List<String> frameworkClasspath() {
    +        String stormHome = System.getProperty("storm.home");
    +
    +        File stormLibDir = new File(stormHome, "lib");
    +        String stormConfDir =
    +                System.getenv("STORM_CONF_DIR") != null ?
    +                System.getenv("STORM_CONF_DIR") :
    +                new File(stormHome, "conf").getAbsolutePath();
    +        File stormExtlibDir = new File(stormHome, "extlib");
    +        String extcp = System.getenv("STORM_EXT_CLASSPATH");
    +        List<String> pathElements = new LinkedList<>();
    +        pathElements.addAll(getFullJars(stormLibDir));
    +        pathElements.addAll(getFullJars(stormExtlibDir));
    +        pathElements.add(extcp);
    +        pathElements.add(stormConfDir);
    +
    +        return pathElements;
    +    }
    +    
    +    @SuppressWarnings("unchecked")
    +    private List<String> asStringList(Object o) {
    +        if (o instanceof String) {
    +            return Arrays.asList((String)o);
    +        } else if (o instanceof List) {
    +            return (List<String>)o;
    +        }
    +        return Collections.EMPTY_LIST;
    +    }
    +    
    +    /**
    +     * Compute the classpath for the worker process
    +     * @param stormJar the topology jar
    +     * @param dependencyLocations any dependencies from the topology
    +     * @return the full classpath
    +     */
    +    protected String getWorkerClassPath(String stormJar, List<String> dependencyLocations) {
    +        List<String> workercp = new ArrayList<>();
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH_BEGINNING)));
    +        workercp.addAll(frameworkClasspath());
    +        workercp.add(stormJar);
    +        workercp.addAll(dependencyLocations);
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH)));
    +        return CPJ.join(workercp);
    +    }
    +
    +    private String substituteChildOptsInternal(String string, int memOnheap) {
    +        if (StringUtils.isNotBlank(string)) {
    +            String p = String.valueOf(_port);
    +            string = string.replace("%ID%", p);
    +            string = string.replace("%WORKER-ID%", _workerId);
    +            string = string.replace("%TOPOLOGY-ID%", _topologyId);
    +            string = string.replace("%WORKER-PORT%", p);
    +            if (memOnheap > 0) {
    +                string = string.replace("%HEAP-MEM%", String.valueOf(memOnheap));
    +            }
    +        }
    +        return string;
    +    }
    +    
    +    protected List<String> substituteChildopts(Object value) {
    +        return substituteChildopts(value, -1);
    +    }
    +
    +    protected List<String> substituteChildopts(Object value, int memOnheap) {
    +        List<String> rets = new ArrayList<>();
    +        if (value instanceof String) {
    +            String string = substituteChildOptsInternal((String) value, memOnheap);
    +            if (StringUtils.isNotBlank(string)) {
    +                String[] strings = string.split("\\s+");
    +                rets.addAll(Arrays.asList(strings));
    +            }
    +        } else if (value instanceof List) {
    +            @SuppressWarnings("unchecked")
    +            List<String> objects = (List<String>) value;
    +            for (String object : objects) {
    +                String str = substituteChildOptsInternal(object, memOnheap);
    +                if (StringUtils.isNotBlank(str)) {
    +                    rets.add(str);
    +                }
    +            }
    +        }
    +        return rets;
    +    }
    +
    +    /**
    +     * Launch the worker process (non-blocking)
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param processExitcallback
    +     *            a callback for when the process exits
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     */
    +    protected void launchWorkerProcess(List<String> command, Map<String, String> env, String logPrefix,
    +            ExitCodeCallback processExitCallback, File targetDir) throws IOException {
    +        SupervisorUtils.launchProcess(command, env, logPrefix, processExitCallback, targetDir);
    +    }
    +
    +    private String getWorkerLoggingConfigFile(String stormHome) {
    +        String log4jConfigurationDir = (String) (_conf.get(Config.STORM_LOG4J2_CONF_DIR));
    +
    +        if (StringUtils.isNotBlank(log4jConfigurationDir)) {
    +            if (!Utils.isAbsolutePath(log4jConfigurationDir)) {
    +                log4jConfigurationDir = stormHome + Utils.FILE_PATH_SEPARATOR + log4jConfigurationDir;
    +            }
    +        } else {
    +            log4jConfigurationDir = stormHome + Utils.FILE_PATH_SEPARATOR + "log4j2";
    +        }
    +        
    +
    +        if (Utils.IS_ON_WINDOWS && !log4jConfigurationDir.startsWith("file:")) {
    +            log4jConfigurationDir = "file:///" + log4jConfigurationDir;
    +        }
    +        return log4jConfigurationDir + Utils.FILE_PATH_SEPARATOR + "worker.xml";
    +    }
    +    
    +    /**
    +     * Get parameters for the class path of the worker process.  Also used by the
    +     * log Writer
    +     * @param stormRoot the root dist dir for the topology
    +     * @return the classpath for the topology as command line arguments.
    +     * @throws IOException on any error.
    +     */
    +    private List<String> getClassPathParams(final String stormRoot) throws IOException {
    +        final String stormJar = ConfigUtils.supervisorStormJarPath(stormRoot);
    +        final StormTopology stormTopology = ConfigUtils.readSupervisorTopology(_conf, _topologyId, _ops);
    +        final List<String> dependencyLocations = new ArrayList<>();
    +        if (stormTopology.get_dependency_jars() != null) {
    +            for (String dependency : stormTopology.get_dependency_jars()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +
    +        if (stormTopology.get_dependency_artifacts() != null) {
    +            for (String dependency : stormTopology.get_dependency_artifacts()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +        final String workerClassPath = getWorkerClassPath(stormJar, dependencyLocations);
    +        
    +        List<String> classPathParams = new ArrayList<>();
    +        classPathParams.add("-cp");
    +        classPathParams.add(workerClassPath);
    +        return classPathParams;
    +    }
    +    
    +    /**
    +     * Get a set of java properties that are common to both the log writer and the worker processes.
    +     * These are mostly system properties that are used by logging.
    +     * @return a list of command line options
    +     */
    +    private List<String> getCommonParams() {
    +        final String stormHome = ConfigUtils.concatIfNotNull(System.getProperty("storm.home"));
    +        final String workersArtifacts = ConfigUtils.workerArtifactsRoot(_conf);
    +        String stormLogDir = ConfigUtils.getLogDir();
    +        String log4jConfigurationFile = getWorkerLoggingConfigFile(stormHome);
    +        
    +        List<String> commonParams = new ArrayList<>();
    +        commonParams.add("-Dlogfile.name=worker.log");
    +        commonParams.add("-Dstorm.home=" + stormHome);
    +        commonParams.add("-Dworkers.artifacts=" + workersArtifacts);
    +        commonParams.add("-Dstorm.id=" + _topologyId);
    +        commonParams.add("-Dworker.id=" + _workerId);
    +        commonParams.add("-Dworker.port=" + _port);
    +        commonParams.add("-Dstorm.log.dir=" + stormLogDir);
    +        commonParams.add("-Dlog4j.configurationFile=" + log4jConfigurationFile);
    +        commonParams.add("-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector");
    +        return commonParams;
    +    }
    +    
    +    private int getMemOnHeap(WorkerResources resources) {
    +        int memOnheap = 0;
    +        if (resources != null && resources.is_set_mem_on_heap() && 
    +                resources.get_mem_on_heap() > 0) {
    +            memOnheap = (int) Math.ceil(resources.get_mem_on_heap());
    +        } else {
    +            // set the default heap memory size for supervisor-test
    +            memOnheap = Utils.getInt(_topoConf.get(Config.WORKER_HEAP_MEMORY_MB), 768);
    +        }
    +        return memOnheap;
    +    }
    +    
    +    private List<String> getWorkerProfilerChildOpts(int memOnheap) {
    +        List<String> workerProfilerChildopts = new ArrayList<>();
    +        if (Utils.getBoolean(_conf.get(Config.WORKER_PROFILER_ENABLED), false)) {
    +            workerProfilerChildopts = substituteChildopts(_conf.get(Config.WORKER_PROFILER_CHILDOPTS), memOnheap);
    +        }
    +        return workerProfilerChildopts;
    +    }
    +    
    +    /**
    +     * a or b the first one that is not null
    +     * @param a something
    +     * @param b something else
    +     * @return a or b the first one that is not null
    +     */
    +    private <V> V OR(V a, V b) {
    --- End diff --
    
    If others need it we can move it then.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @abellina I addressed all of your comments.  I also have manually run all of the unit tests and they are passing for me, despite the travis issues.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76479255
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ContainerLauncher.java ---
    @@ -0,0 +1,99 @@
    +/**
    + * 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.Map;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Launches containers
    + */
    +public abstract class ContainerLauncher {
    +    private static final Logger LOG = LoggerFactory.getLogger(ContainerLauncher.class);
    +    
    +    /**
    +     * Factory to create the right container launcher 
    +     * for the config and the environment.
    +     * @param conf the config
    +     * @param supervisorId the ID of the supervisor
    +     * @param sharedContext Used in local mode to let workers talk together without netty
    +     * @return the proper container launcher
    +     * @throws IOException on any error
    +     */
    +    public static ContainerLauncher mk(Map<String, Object> conf, String supervisorId, IContext sharedContext) throws IOException {
    +        if (ConfigUtils.isLocalMode(conf)) {
    +            return new LocalContainerLauncher(conf, supervisorId, sharedContext);
    +        }
    +        
    +        ResourceIsolationInterface resourceIsolationManager = null;
    +        if (Utils.getBoolean(conf.get(Config.STORM_RESOURCE_ISOLATION_PLUGIN_ENABLE), false)) {
    +            resourceIsolationManager = Utils.newInstance((String) conf.get(Config.STORM_RESOURCE_ISOLATION_PLUGIN));
    --- End diff --
    
    It seems `Config.STORM_RESOURCE_ISOLATION_PLUGIN` should have been declared as a String, but is an Object.  There's no harm, but it may be worth fixing it now.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77364277
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/RunAsUserContainer.java ---
    @@ -0,0 +1,107 @@
    +/**
    + * 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.File;
    +import java.io.IOException;
    +import java.util.Arrays;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class RunAsUserContainer extends BasicContainer {
    +    private static final Logger LOG = LoggerFactory.getLogger(RunAsUserContainer.class);
    +
    +    public RunAsUserContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState state, ResourceIsolationInterface resourceIsolationManager, boolean recover) throws IOException {
    +        super(port, assignment, conf, supervisorId, state, resourceIsolationManager, recover);
    +        if (Utils.isOnWindows()) {
    +            throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +        }
    +    }
    +
    +    public RunAsUserContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        super(workerId, conf, supervisorId, resourceIsolationManager);
    +        if (Utils.isOnWindows()) {
    +            throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +        }
    +    }
    +
    +    private void signal(long pid, int signal) throws IOException {
    +        List<String> commands = Arrays.asList("signal", String.valueOf(pid), String.valueOf(signal));
    +        String user = getWorkerUser();
    +        String logPrefix = "kill -"+signal+" " + pid;
    +        SupervisorUtils.processLauncherAndWait(_conf, user, commands, null, logPrefix);
    +    }
    +    
    +    @Override
    +    protected void kill(long pid) throws IOException {
    +        signal(pid, 15);
    +    }
    +    
    +    @Override
    +    protected void forceKill(long pid) throws IOException {
    +        signal(pid, 9);
    +    }
    +    
    +    @Override
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix, File targetDir) throws IOException, InterruptedException {
    +        String user = this.getWorkerUser();
    +        String td = targetDir.getAbsolutePath();
    +        LOG.info("Running as user:{} command:{}", user, command);
    +        String containerFile = Utils.containerFilePath(td);
    +        if (Utils.checkFileExists(containerFile)) {
    +            SupervisorUtils.rmrAsUser(_conf, containerFile, containerFile);
    +        }
    +        String scriptFile = Utils.scriptFilePath(td);
    +        if (Utils.checkFileExists(scriptFile)) {
    +            SupervisorUtils.rmrAsUser(_conf, scriptFile, scriptFile);
    +        }
    +        String script = Utils.writeScript(td, command, env);
    +        List<String> args = Arrays.asList("profiler", td, script);
    +        int ret = SupervisorUtils.processLauncherAndWait(_conf, user, args, env, logPrefix);
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public List<String> updateCommandForIsolation(List<String> command) {
    +        //With run as user we do the isolation before switching users, so don't modify the original
    +        // command line
    +        return command;
    +    }   
    --- End diff --
    
    @revans2 , Let's move updateCommandForIsolation into launchWorkerProcess, as we discussed. It will clean up the code.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @knusbaum @srdo @harshach @HeartSaVioR @abellina I really would appreciate it if you could take another look at this patch.  I think everything is ready to go except a final squash before merging it.  The unit tests all pass, I have run some basic manual tests with CGroups and run as user enabled.  I have run a number of manual tests on my mac, including rebalancing, killing the workers, killing the supervisor, and going back and forth between old and new supervisor.  I have manually deleted and added files/directories in storm-dist on the supervisor.  I also ran the full set of the internal yahoo integration tests against it and everything has passed.
    
    I really would like to start rolling this out internally at yahoo, but I want to get as many eyes looking at it to find issues first.  Better here then in production.  On the upside the bugs I have found have been rather easy to debug, and very reproducible because the race conditions have been eliminated when interacting with the Container.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76308994
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,749 @@
    +/**
    + * 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,
    +        WATING_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 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 monitorFreqMs,
    +                ContainerLauncher containerLauncher, String host, int port,
    +                ISupervisor iSupervisor, LocalState localState) {
    +            this.localizer = localizer;
    +            this.hbTimeoutMs = hbTimeoutMs;
    +            this.firstHbTimeoutMs = firstHbTimeoutMs;
    +            this.containerLauncher = containerLauncher;
    +            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 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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WATING_FOR_WORKER_START) {
    +                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 equivilant(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;
    +                    } else 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 WATING_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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), staticState.port);
    +        }
    +        Time.sleep(1000);
    --- End diff --
    
    You are right that I missed this.
    
    https://github.com/apache/storm/blob/master/storm-core/src/jvm/org/apache/storm/daemon/supervisor/workermanager/DefaultWorkerManager.java#L268
    
    I'll fix it.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78991476
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,769 @@
    +/**
    + * 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);
    +            }
    +            if (container != null) {
    +                sb.append(" container: ");
    +                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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WAITING_FOR_WORKER_START) {
    +                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 equivilant(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;
    +                    } else 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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), 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.get_topology_id(), 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);
    +        
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivilant(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +        
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for WAITING_FOR_WORKER_START state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleWaitingForWorkerStart(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb != null) {
    +            long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +            if (hbAgeMs <= staticState.hbTimeoutMs) {
    +                return dynamicState.withState(MachineState.RUNNING);
    +            }
    +        }
    +        
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            //We were rescheduled while waiting for the worker to come up
    +            return Slot.killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffms = (Time.currentTimeMillis() - dynamicState.startTime);
    +        if (timeDiffms > staticState.firstHbTimeoutMs) {
    +            LOG.warn("SLOT {}: Container {} failed to launch in {} ms.", staticState.port, dynamicState.container, staticState.firstHbTimeoutMs);
    +            dynamicState.container.kill();
    +            Time.sleep(staticState.killSleepMs);
    +            return dynamicState.withState(MachineState.KILL_AND_RELAUNCH);
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for RUNNING state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleRunning(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            LOG.warn("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAssignment, dynamicState.newAssignment);
    +            //Scheduling changed while running...
    +            return killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        if (dynamicState.container.didMainProcessExit()) {
    +            LOG.warn("SLOT {}: main process has exited", staticState.port);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb == null) {
    +            LOG.warn("SLOT {}: HB returned as null", staticState.port);
    +            //This should never happen, but to be safe
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +        if (timeDiffMs > staticState.hbTimeoutMs) {
    +            LOG.warn("SLOT {}: HB is too old {} > {}", staticState.port, timeDiffMs, staticState.hbTimeoutMs);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        //The worker is up and running check for profiling requests
    +        if (!dynamicState.profileActions.isEmpty()) {
    +            HashSet<TopoProfileAction> mod = new HashSet<>(dynamicState.profileActions);
    +            HashSet<TopoProfileAction> modPending = new HashSet<>(dynamicState.pendingStopProfileActions);
    +            Iterator<TopoProfileAction> iter = mod.iterator();
    +            while (iter.hasNext()) {
    +                TopoProfileAction action = iter.next();
    +                if (!action.topoId.equals(dynamicState.currentAssignment.get_topology_id())) {
    +                    iter.remove();
    +                    LOG.warn("Dropping {} wrong topology is running", action);
    +                    //Not for this topology so skip it
    +                } else {
    +                    if (modPending.contains(action)) {
    +                        boolean isTimeForStop = Time.currentTimeMillis() > action.request.get_time_stamp();
    +                        if (isTimeForStop) {
    +                            if (dynamicState.container.runProfiling(action.request, true)) {
    +                                LOG.debug("Stopped {} action finished", action);
    +                                iter.remove();
    +                                modPending.remove(action);
    +                            } else {
    +                                LOG.warn("Stopping {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            LOG.debug("Still pending {} now: {}", action, Time.currentTimeMillis());
    +                        }
    +                    } else {
    +                        //J_PROFILE_START is not used.  When you see a J_PROFILE_STOP
    +                        // start profiling and save it away to stop when timeout happens
    +                        if (action.request.get_action() == ProfileAction.JPROFILE_STOP) {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                modPending.add(action);
    +                                LOG.debug("Started {} now: {}", action, Time.currentTimeMillis());
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                LOG.debug("Started {} action finished", action);
    +                                iter.remove();
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        }
    +                    }
    +                }
    +            }
    +            dynamicState = dynamicState.withProfileActions(mod, modPending);
    +        }
    +        Time.sleep(staticState.monitorFreqMs);
    +        return dynamicState;
    +    }
    +
    +    static DynamicState handleEmpty(DynamicState dynamicState, StaticState staticState) throws InterruptedException {
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +        }
    +        //Both assignments are null, just wait
    +        if (dynamicState.profileActions != null && !dynamicState.profileActions.isEmpty()) {
    +            //Nothing is scheduled here so throw away all of the profileActions
    +            LOG.warn("Dropping {} no topology is running", dynamicState.profileActions);
    +            dynamicState = dynamicState.withProfileActions(Collections.<TopoProfileAction> emptySet(), Collections.<TopoProfileAction> emptySet());
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +    
    +    private final AtomicReference<LocalAssignment> newAssignment = new AtomicReference<>();
    +    private final AtomicReference<Set<TopoProfileAction>> profiling =
    +            new AtomicReference<Set<TopoProfileAction>>(new HashSet<TopoProfileAction>());
    +    private final StaticState staticState;
    +    private final IStormClusterState clusterState;
    +    private volatile boolean done = false;
    +    private volatile DynamicState dynamicState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignmants;
    +    
    +    public Slot(ILocalizer localizer, Map<String, Object> conf, 
    +            ContainerLauncher containerLauncher, String host,
    +            int port, LocalState localState,
    +            IStormClusterState clusterState,
    +            ISupervisor iSupervisor,
    +            AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignmants) throws Exception {
    +        super("SLOT_"+port);
    +
    +        this.cachedCurrentAssignmants = cachedCurrentAssignmants;
    +        this.clusterState = clusterState;
    +        Map<Integer, LocalAssignment> assignments = localState.getLocalAssignmentsMap();
    +        LocalAssignment currentAssignment = null;
    +        if (assignments != null) {
    +            currentAssignment = assignments.get(port);
    +        }
    +        Container container = null;
    +        if (currentAssignment != null) { 
    +            container = containerLauncher.recoverContainer(port, currentAssignment, localState);
    +        }
    +        
    +        LocalAssignment newAssignment = currentAssignment;
    +        if (currentAssignment != null && container == null) {
    +            currentAssignment = null;
    +            //Assigned something but it is not running
    +        }
    +        
    +        dynamicState = new DynamicState(currentAssignment, container, newAssignment);
    +        staticState = new StaticState(localizer, 
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_START_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_MONITOR_FREQUENCY_SECS)) * 1000,
    +                containerLauncher,
    +                host,
    +                port,
    +                iSupervisor,
    +                localState);
    +        this.newAssignment.set(dynamicState.newAssignment);
    +        if (MachineState.RUNNING == dynamicState.state) {
    +            //We are running so we should recover the blobs.
    +            staticState.localizer.recoverRunningTopology(currentAssignment.get_topology_id(), port);
    +        }
    +        LOG.warn("SLOT {}:{} Starting in state {} - assignment {}", staticState.host, staticState.port, dynamicState.state, dynamicState.currentAssignment);
    +    }
    +    
    +    public MachineState getMachineState() {
    +        return dynamicState.state;
    +    }
    +    
    +    /**
    +     * Set a new assignment asynchronously
    +     * @param newAssignment the new assignment for this slot to run, null to run nothing
    +     */
    +    public void setNewAssignment(LocalAssignment newAssignment) {
    +        this.newAssignment.set(newAssignment);
    +    }
    +    
    +    public void addProfilerActions(Set<TopoProfileAction> actions) {
    +        if (actions != null) {
    +            while(true) {
    +                Set<TopoProfileAction> orig = profiling.get();
    +                Set<TopoProfileAction> newActions = new HashSet<>(orig);
    +                newActions.addAll(actions);
    +                if (profiling.compareAndSet(orig, newActions)) {
    +                    return;
    +                }
    +            }
    +        }
    +    }
    +    
    +    public void run() {
    +        try {
    +            while(!done) {
    +                LocalAssignment localNewAssignment = newAssignment.get();
    +                Set<TopoProfileAction> origProfileActions = new HashSet<>(profiling.get());
    +                Set<TopoProfileAction> removed = new HashSet<>(origProfileActions);
    +                
    +                DynamicState nextState = 
    +                        stateMachineStep(dynamicState.withNewAssignment(localNewAssignment)
    +                                .withProfileActions(origProfileActions, dynamicState.pendingStopProfileActions), staticState);
    +
    +                if (LOG.isDebugEnabled() || dynamicState.state != nextState.state) {
    +                    LOG.info("STATE {} -> {}", dynamicState, nextState);
    +                }
    +                //Save the current state for recovery
    +                if (!equivilant(nextState.currentAssignment, dynamicState.currentAssignment)) {
    +                    LOG.warn("SLOT {}: Changing current assignment from {} to {}", staticState.port, dynamicState.currentAssignment, nextState.currentAssignment);
    +                    synchronized(staticState.localState) {
    +                        Map<Integer, LocalAssignment> assignments = staticState.localState.getLocalAssignmentsMap();
    +                        if (assignments == null) {
    +                            assignments = new HashMap<>();
    +                        }
    +                        if (nextState.currentAssignment == null) {
    +                            assignments.remove(staticState.port);
    +                        } else {
    +                            assignments.put(staticState.port, nextState.currentAssignment);
    +                        }
    +                        staticState.localState.setLocalAssignmentsMap(assignments);
    +                    }
    +                    Map<Long, LocalAssignment> update = null;
    +                    Map<Long, LocalAssignment> orig = null;
    +                    do {
    +                        Long lport = new Long(staticState.port);
    +                        orig = cachedCurrentAssignmants.get();
    +                        update = new HashMap<>(orig);
    +                        if (nextState.currentAssignment == null) {
    +                            update.remove(lport);
    +                        } else {
    +                            update.put(lport, nextState.currentAssignment);
    +                        }
    +                    } while (!cachedCurrentAssignmants.compareAndSet(orig, update));
    +                }
    +                
    +                // clean up the profiler actions that are not being processed
    +                removed.removeAll(dynamicState.profileActions);
    +                removed.removeAll(dynamicState.pendingStopProfileActions);
    +                for (TopoProfileAction action: removed) {
    +                    try {
    +                        clusterState.deleteTopologyProfileRequests(action.topoId, action.request);
    +                    } catch (Exception e) {
    +                        LOG.error("Error trying to remove profiling request, it will be retried", e);
    +                    }
    +                }
    +                Set<TopoProfileAction> orig, copy;
    +                do {
    +                    orig = profiling.get();
    +                    copy = new HashSet<>(orig);
    +                    copy.removeAll(removed);
    +                } while (!profiling.compareAndSet(orig, copy));
    +                dynamicState = nextState;
    +                newAssignment.compareAndSet(localNewAssignment, dynamicState.newAssignment);
    --- End diff --
    
    I don't really understand this assignment. The localNewAssignment set in L726 is in L731 put into a DynamicState. The resulting state from doing a step in the state machine is then used to overwrite dynamicState in L759. I can't find anywhere dynamicState.newAssignment is changed in the state machine, so I don't understand how localNewAssignment and dynamicState.newAssignment can be different. Where/when does nextState have its newAssignment changed?


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76273872
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,749 @@
    +/**
    + * 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,
    +        WATING_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 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 monitorFreqMs,
    +                ContainerLauncher containerLauncher, String host, int port,
    +                ISupervisor iSupervisor, LocalState localState) {
    +            this.localizer = localizer;
    +            this.hbTimeoutMs = hbTimeoutMs;
    +            this.firstHbTimeoutMs = firstHbTimeoutMs;
    +            this.containerLauncher = containerLauncher;
    +            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 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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WATING_FOR_WORKER_START) {
    +                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 equivilant(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;
    +                    } else 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 WATING_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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), staticState.port);
    +        }
    +        Time.sleep(1000);
    +        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(1000);
    +        
    +        //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.get_topology_id(), 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);
    +        
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivilant(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WATING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +        
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WATING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    --- End diff --
    
    the value here should be driven by config


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77416935
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    +                            if (actions == null) {
    +                                actions = new HashSet<>();
    +                                filtered.put(port.intValue(), actions);
    +                            }
    +                            actions.add(new TopoProfileAction(topoId, req));
    +                        }
    +                    }
    +                }
    +            }
    +            
    +            for (Integer port: allPorts) {
    +                Slot slot = slots.get(port);
    +                if (slot == null) {
    +                    slot = mkSlot(port);
    +                    slots.put(port, slot);
    +                }
    +                slot.setNewAssignment(allAssignments.get(port));
    +                slot.addProfilerActions(filtered.get(port));
    +            }
    +            
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +    }
    +    
    +    protected Map<String, VersionedData<Assignment>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +            Map<String, VersionedData<Assignment>> localAssignmentVersion, Runnable callback) throws Exception {
    +        Map<String, VersionedData<Assignment>> updateAssignmentVersion = new HashMap<>();
    +        for (String topoId : topoIds) {
    +            Integer recordedVersion = -1;
    +            Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +            VersionedData<Assignment> locAssignment = localAssignmentVersion.get(topoId);
    +            if (locAssignment != null) {
    +                recordedVersion = locAssignment.getVersion();
    +            }
    +            if (version == null) {
    +                // ignore
    +            } else if (version == recordedVersion) {
    +                updateAssignmentVersion.put(topoId, locAssignment);
    +            } else {
    +                VersionedData<Assignment> assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                updateAssignmentVersion.put(topoId, assignmentVersion);
    +            }
    +        }
    +        return updateAssignmentVersion;
    +    }
    +    
    +    protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +        Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +        for (String stormId : stormIds) {
    +            List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +            ret.put(stormId, profileRequests);
    +        }
    +        return ret;
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readAssignments(Map<String, VersionedData<Assignment>> assignmentsSnapshot,
    +            String assignmentId, AtomicInteger retries) {
    +        try {
    +            Map<Integer, LocalAssignment> portLA = new HashMap<Integer, LocalAssignment>();
    +            for (Map.Entry<String, VersionedData<Assignment>> assignEntry : assignmentsSnapshot.entrySet()) {
    +                String topoId = assignEntry.getKey();
    +                Assignment assignment = assignEntry.getValue().getData();
    +
    +                Map<Integer, LocalAssignment> portTasks = readMyExecutors(topoId, assignmentId, assignment);
    +
    +                for (Map.Entry<Integer, LocalAssignment> entry : portTasks.entrySet()) {
    +
    +                    Integer port = entry.getKey();
    +
    +                    LocalAssignment la = entry.getValue();
    +
    +                    if (!portLA.containsKey(port)) {
    +                        portLA.put(port, la);
    +                    } else {
    +                        throw new RuntimeException("Should not have multiple topologies assigned to one port");
    +                    }
    +                }
    +            }
    +            retries.set(0);
    +            return portLA;
    +        } catch (RuntimeException e) {
    +            if (retries.get() > 2) {
    +                throw e;
    +            } else {
    +                retries.addAndGet(1);
    +            }
    +            LOG.warn("{} : retrying {} of 3", e.getMessage(), retries.get());
    --- End diff --
    
    This is code that came from clojure.  It might be worth rewriting the entire function to not use the AtomicInteger, but I thought it would be best to leave the code alone, and if we wanted to clean it up later we could.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77403662
  
    --- 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);
    +            }
    +            if (container != null) {
    +                sb.append(" container: ");
    +                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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WAITING_FOR_WORKER_START) {
    +                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;
    +                    } else 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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), 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.get_topology_id(), 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);
    +        
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        //The child processes typically exit in < 1 sec.  If 2 mins later they are still around something is wrong
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for WAITING_FOR_WORKER_START state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleWaitingForWorkerStart(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb != null) {
    +            long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +            if (hbAgeMs <= staticState.hbTimeoutMs) {
    +                return dynamicState.withState(MachineState.RUNNING);
    +            }
    +        }
    +        
    +        if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            //We were rescheduled while waiting for the worker to come up
    +            return Slot.killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffms = (Time.currentTimeMillis() - dynamicState.startTime);
    +        if (timeDiffms > staticState.firstHbTimeoutMs) {
    +            LOG.warn("SLOT {}: Container {} failed to launch in {} ms.", staticState.port, dynamicState.container, staticState.firstHbTimeoutMs);
    +            dynamicState.container.kill();
    +            Time.sleep(staticState.killSleepMs);
    +            return dynamicState.withState(MachineState.KILL_AND_RELAUNCH);
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for RUNNING state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleRunning(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            LOG.warn("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAssignment, dynamicState.newAssignment);
    +            //Scheduling changed while running...
    +            return killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        if (dynamicState.container.didMainProcessExit()) {
    +            LOG.warn("SLOT {}: main process has exited", staticState.port);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb == null) {
    +            LOG.warn("SLOT {}: HB returned as null", staticState.port);
    +            //This should never happen, but to be safe
    --- End diff --
    
    As we discussed, this could happen in the case when we successfully recover a container while the worker it contains never initialized far enough to write its first heartbeat. This is still an OK state to be in, so the comment should change.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77150344
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +
    +    //For testing purposes
    +    public BasicContainer(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager, LocalState localState,
    +            String profileCmd) throws IOException {
    +        super(ops, port, assignment, conf, topoConf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +        _profileCmd = profileCmd;
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState localState, ResourceIsolationInterface resourceIsolationManager, boolean recover)
    +            throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                LOG.info("Recovered Worker {}", wid);
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    --- End diff --
    
    Could this check if it is called at the wrong time? For example by throwing an IllegalStateException if _workerId is not null?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77144475
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/LocalContainer.java ---
    @@ -0,0 +1,85 @@
    +/**
    + * 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.Map;
    +
    +import org.apache.storm.ProcessSimulator;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import clojure.java.api.Clojure;
    +import clojure.lang.IFn;
    +
    +public class LocalContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(LocalContainer.class);
    +    private volatile boolean _isAlive = false;
    +    private final IContext _sharedContext;
    +    
    +    public LocalContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId, IContext sharedContext) throws IOException {
    +        super(port, assignment, conf, supervisorId, null);
    +        _sharedContext = sharedContext;
    +        _workerId = Utils.uuid();
    +    }
    +    
    +    @Override
    +    public void launch() throws IOException {
    +        IFn mkWorker = Clojure.var("org.apache.storm.daemon.worker", "mk-worker");
    --- End diff --
    
    Let's leave a comment that it should be modified when worker is being ported.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76874493
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,769 @@
    +/**
    + * 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);
    +            }
    +            if (container != null) {
    +                sb.append(" container: ");
    +                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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WAITING_FOR_WORKER_START) {
    +                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 equivilant(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;
    +                    } else 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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), 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.get_topology_id(), 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);
    +        
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivilant(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +        
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for WAITING_FOR_WORKER_START state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleWaitingForWorkerStart(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb != null) {
    +            long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +            if (hbAgeMs <= staticState.hbTimeoutMs) {
    +                return dynamicState.withState(MachineState.RUNNING);
    +            }
    +        }
    +        
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            //We were rescheduled while waiting for the worker to come up
    +            return Slot.killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffms = (Time.currentTimeMillis() - dynamicState.startTime);
    +        if (timeDiffms > staticState.firstHbTimeoutMs) {
    +            LOG.warn("SLOT {}: Container {} failed to launch in {} ms.", staticState.port, dynamicState.container, staticState.firstHbTimeoutMs);
    +            dynamicState.container.kill();
    +            Time.sleep(staticState.killSleepMs);
    +            return dynamicState.withState(MachineState.KILL_AND_RELAUNCH);
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for RUNNING state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleRunning(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            LOG.warn("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAssignment, dynamicState.newAssignment);
    +            //Scheduling changed while running...
    +            return killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        if (dynamicState.container.didMainProcessExit()) {
    +            LOG.warn("SLOT {}: main process has exited", staticState.port);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb == null) {
    +            LOG.warn("SLOT {}: HB returned as null", staticState.port);
    +            //This should never happen, but to be safe
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +        if (timeDiffMs > staticState.hbTimeoutMs) {
    +            LOG.warn("SLOT {}: HB is too old {} > {}", staticState.port, timeDiffMs, staticState.hbTimeoutMs);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        //The worker is up and running check for profiling requests
    +        if (!dynamicState.profileActions.isEmpty()) {
    +            HashSet<TopoProfileAction> mod = new HashSet<>(dynamicState.profileActions);
    +            HashSet<TopoProfileAction> modPending = new HashSet<>(dynamicState.pendingStopProfileActions);
    +            Iterator<TopoProfileAction> iter = mod.iterator();
    +            while (iter.hasNext()) {
    +                TopoProfileAction action = iter.next();
    +                if (!action.topoId.equals(dynamicState.currentAssignment.get_topology_id())) {
    +                    iter.remove();
    +                    LOG.warn("Dropping {} wrong topology is running", action);
    +                    //Not for this topology so skip it
    +                } else {
    +                    if (modPending.contains(action)) {
    +                        boolean isTimeForStop = Time.currentTimeMillis() > action.request.get_time_stamp();
    +                        if (isTimeForStop) {
    +                            if (dynamicState.container.runProfiling(action.request, true)) {
    +                                LOG.debug("Stopped {} action finished", action);
    +                                iter.remove();
    +                                modPending.remove(action);
    +                            } else {
    +                                LOG.warn("Stopping {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            LOG.debug("Still pending {} now: {}", action, Time.currentTimeMillis());
    +                        }
    +                    } else {
    +                        //J_PROFILE_START is not used.  When you see a J_PROFILE_STOP
    +                        // start profiling and save it away to stop when timeout happens
    +                        if (action.request.get_action() == ProfileAction.JPROFILE_STOP) {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                modPending.add(action);
    +                                LOG.debug("Started {} now: {}", action, Time.currentTimeMillis());
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                LOG.debug("Started {} action finished", action);
    +                                iter.remove();
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        }
    +                    }
    +                }
    +            }
    +            dynamicState = dynamicState.withProfileActions(mod, modPending);
    +        }
    +        Time.sleep(staticState.monitorFreqMs);
    +        return dynamicState;
    +    }
    +
    +    static DynamicState handleEmpty(DynamicState dynamicState, StaticState staticState) throws InterruptedException {
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +        }
    +        //Both assignments are null, just wait
    +        if (dynamicState.profileActions != null && !dynamicState.profileActions.isEmpty()) {
    +            //Nothing is scheduled here so throw away all of the profileActions
    +            LOG.warn("Dropping {} no topology is running", dynamicState.profileActions);
    +            dynamicState = dynamicState.withProfileActions(Collections.<TopoProfileAction> emptySet(), Collections.<TopoProfileAction> emptySet());
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +    
    +    private final AtomicReference<LocalAssignment> newAssignment = new AtomicReference<>();
    +    private final AtomicReference<Set<TopoProfileAction>> profiling =
    +            new AtomicReference<Set<TopoProfileAction>>(new HashSet<TopoProfileAction>());
    +    private final StaticState staticState;
    +    private final IStormClusterState clusterState;
    +    private volatile boolean done = false;
    +    private volatile DynamicState dynamicState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignmants;
    +    
    +    public Slot(ILocalizer localizer, Map<String, Object> conf, 
    +            ContainerLauncher containerLauncher, String host,
    +            int port, LocalState localState,
    +            IStormClusterState clusterState,
    +            ISupervisor iSupervisor,
    +            AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignmants) throws Exception {
    +        super("SLOT_"+port);
    +
    +        this.cachedCurrentAssignmants = cachedCurrentAssignmants;
    +        this.clusterState = clusterState;
    +        Map<Integer, LocalAssignment> assignments = localState.getLocalAssignmentsMap();
    +        LocalAssignment currentAssignment = null;
    +        if (assignments != null) {
    +            currentAssignment = assignments.get(port);
    +        }
    +        Container container = null;
    +        if (currentAssignment != null) { 
    +            container = containerLauncher.recoverContainer(port, currentAssignment, localState);
    +        }
    +        
    +        LocalAssignment newAssignment = currentAssignment;
    +        if (currentAssignment != null && container == null) {
    +            currentAssignment = null;
    +            //Assigned something but it is not running
    +        }
    +        
    +        dynamicState = new DynamicState(currentAssignment, container, newAssignment);
    +        staticState = new StaticState(localizer, 
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_START_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_MONITOR_FREQUENCY_SECS)) * 1000,
    +                containerLauncher,
    +                host,
    +                port,
    +                iSupervisor,
    +                localState);
    +        this.newAssignment.set(dynamicState.newAssignment);
    --- End diff --
    
    We could, but that is meant to be a user facing API, and so I don't see a lot of value in trying to maintain the abstraction here.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78999636
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected final String _stormHome = System.getProperty("storm.home");
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +    
    +    /**
    +     * Create a new BasicContainer
    +     * @param type the type of container being made.
    +     * @param conf the supervisor config
    +     * @param supervisorId the ID of the supervisor this is a part of.
    +     * @param port the port the container is on.  Should be <= 0 if only a partial recovery
    +     * @param assignment the assignment for this container. Should be null if only a partial recovery.
    +     * @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used.
    +     * @param localState the local state of the supervisor.  May be null if partial recovery
    +     * @param workerId the id of the worker to use.  Must not be null if doing a partial recovery.
    +     * @param ops file system operations (mostly for testing) if null a new one is made
    +     * @param topoConf the config of the topology (mostly for testing) if null 
    +     * and not a partial recovery the real conf is read.
    +     * @param profileCmd the command to use when profiling (used for testing)
    +     */
    +    public BasicContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int port,
    --- End diff --
    
    Extremely nitpicky, but if the last 3 parameters are just for testing, consider making a package private overload of this and removing them from the public constructor


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76801694
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,629 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +
    +    //For testing purposes
    +    public BasicContainer(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager, LocalState localState,
    +            String profileCmd) throws IOException {
    +        super(ops, port, assignment, conf, topoConf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +        _profileCmd = profileCmd;
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState localState, ResourceIsolationInterface resourceIsolationManager, boolean recover)
    +            throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException(
    +                    "Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            _localState.setApprovedWorkers(workersToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        setup();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = _ops.slurpString(new File(str)).trim();
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        if (command.size() > 0) {
    +            return runProfilingCommand(command, env, logPrefix, targetFile);
    +        }
    +        LOG.warn("PROFILING REQUEST NOT SUPPORTED {} IGNORED...", request);
    +        return true;
    +    }
    +
    +    /**
    +     * Get the command to run when doing profiling
    +     * @param action the profiling action to perform
    +     * @param stop if this is meant to stop the profiling or start it
    +     * @param workerPid the PID of the process to profile
    +     * @param targetDir the current working directory of the worker process
    +     * @return the command to run for profiling.
    +     */
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        switch(action) {
    +            case JMAP_DUMP:
    +                return jmapDumpCmd(workerPid, targetDir);
    +            case JSTACK_DUMP:
    +                return jstackDumpCmd(workerPid, targetDir);
    +            case JPROFILE_DUMP:
    +                return jprofileDump(workerPid, targetDir);
    +            case JVM_RESTART:
    +                return jprofileJvmRestart(workerPid);
    +            case JPROFILE_STOP:
    +                if (stop) {
    +                    return jprofileStop(workerPid, targetDir);
    +                }
    +                return jprofileStart(workerPid);
    +            default:
    +                return Lists.newArrayList();
    +        }
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +
    +    /**
    +     * Compute the java.library.path that should be used for the worker.
    +     * This helps it to load JNI libraries that are packaged in the uber jar.
    +     * @param stormRoot the root directory of the worker process
    +     * @param conf the config for the supervisor.
    +     * @return the java.library.path/LD_LIBRARY_PATH to use so native libraries load correctly.
    +     */
    +    protected String jlp(String stormRoot, Map<String, Object> conf) {
    +        String resourceRoot = stormRoot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +        String os = System.getProperty("os.name").replaceAll("\\s+", "_");
    +        String arch = System.getProperty("os.arch");
    +        String archResourceRoot = resourceRoot + Utils.FILE_PATH_SEPARATOR + os + "-" + arch;
    +        String ret = CPJ.join(archResourceRoot, resourceRoot,
    +                conf.get(Config.JAVA_LIBRARY_PATH));
    +        return ret;
    +    }
    +
    +    /**
    +     * Returns a collection of jar file names found under the given directory.
    +     * @param dir the directory to search
    +     * @return the jar file names
    +     */
    +    protected List<String> getFullJars(File dir) {
    +        File[] files = dir.listFiles(jarFilter);
    +
    +        if (files == null) {
    +            return Collections.emptyList();
    +        }
    +
    +        return Arrays.stream(files).map(f -> f.getAbsolutePath())
    +                .collect(Collectors.toList());
    +    }
    +    
    +    protected List<String> frameworkClasspath() {
    +        String stormHome = System.getProperty("storm.home");
    +
    +        File stormLibDir = new File(stormHome, "lib");
    +        String stormConfDir =
    +                System.getenv("STORM_CONF_DIR") != null ?
    +                System.getenv("STORM_CONF_DIR") :
    +                new File(stormHome, "conf").getAbsolutePath();
    +        File stormExtlibDir = new File(stormHome, "extlib");
    +        String extcp = System.getenv("STORM_EXT_CLASSPATH");
    +        List<String> pathElements = new LinkedList<>();
    +        pathElements.addAll(getFullJars(stormLibDir));
    +        pathElements.addAll(getFullJars(stormExtlibDir));
    +        pathElements.add(extcp);
    +        pathElements.add(stormConfDir);
    +
    +        return pathElements;
    +    }
    +    
    +    @SuppressWarnings("unchecked")
    +    private List<String> asStringList(Object o) {
    +        if (o instanceof String) {
    +            return Arrays.asList((String)o);
    +        } else if (o instanceof List) {
    +            return (List<String>)o;
    +        }
    +        return Collections.EMPTY_LIST;
    +    }
    +    
    +    /**
    +     * Compute the classpath for the worker process
    +     * @param stormJar the topology jar
    +     * @param dependencyLocations any dependencies from the topology
    +     * @return the full classpath
    +     */
    +    protected String getWorkerClassPath(String stormJar, List<String> dependencyLocations) {
    +        List<String> workercp = new ArrayList<>();
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH_BEGINNING)));
    +        workercp.addAll(frameworkClasspath());
    +        workercp.add(stormJar);
    +        workercp.addAll(dependencyLocations);
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH)));
    +        return CPJ.join(workercp);
    +    }
    +
    +    private String substituteChildOptsInternal(String string, int memOnheap) {
    +        if (StringUtils.isNotBlank(string)) {
    +            String p = String.valueOf(_port);
    +            string = string.replace("%ID%", p);
    +            string = string.replace("%WORKER-ID%", _workerId);
    +            string = string.replace("%TOPOLOGY-ID%", _topologyId);
    +            string = string.replace("%WORKER-PORT%", p);
    +            if (memOnheap > 0) {
    +                string = string.replace("%HEAP-MEM%", String.valueOf(memOnheap));
    +            }
    +        }
    +        return string;
    +    }
    +    
    +    protected List<String> substituteChildopts(Object value) {
    +        return substituteChildopts(value, -1);
    +    }
    +
    +    protected List<String> substituteChildopts(Object value, int memOnheap) {
    +        List<String> rets = new ArrayList<>();
    +        if (value instanceof String) {
    +            String string = substituteChildOptsInternal((String) value, memOnheap);
    +            if (StringUtils.isNotBlank(string)) {
    +                String[] strings = string.split("\\s+");
    +                rets.addAll(Arrays.asList(strings));
    +            }
    +        } else if (value instanceof List) {
    +            @SuppressWarnings("unchecked")
    +            List<String> objects = (List<String>) value;
    +            for (String object : objects) {
    +                String str = substituteChildOptsInternal(object, memOnheap);
    +                if (StringUtils.isNotBlank(str)) {
    +                    rets.add(str);
    +                }
    +            }
    +        }
    +        return rets;
    +    }
    +
    +    /**
    +     * Launch the worker process (non-blocking)
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param processExitcallback
    +     *            a callback for when the process exits
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     */
    +    protected void launchWorkerProcess(List<String> command, Map<String, String> env, String logPrefix,
    +            ExitCodeCallback processExitCallback, File targetDir) throws IOException {
    +        SupervisorUtils.launchProcess(command, env, logPrefix, processExitCallback, targetDir);
    +    }
    +
    +    private String getWorkerLoggingConfigFile(String stormHome) {
    +        String log4jConfigurationDir = (String) (_conf.get(Config.STORM_LOG4J2_CONF_DIR));
    +
    +        if (StringUtils.isNotBlank(log4jConfigurationDir)) {
    +            if (!Utils.isAbsolutePath(log4jConfigurationDir)) {
    +                log4jConfigurationDir = stormHome + Utils.FILE_PATH_SEPARATOR + log4jConfigurationDir;
    +            }
    +        } else {
    +            log4jConfigurationDir = stormHome + Utils.FILE_PATH_SEPARATOR + "log4j2";
    +        }
    +        
    +
    +        if (Utils.IS_ON_WINDOWS && !log4jConfigurationDir.startsWith("file:")) {
    +            log4jConfigurationDir = "file:///" + log4jConfigurationDir;
    +        }
    +        return log4jConfigurationDir + Utils.FILE_PATH_SEPARATOR + "worker.xml";
    +    }
    +    
    +    /**
    +     * Get parameters for the class path of the worker process.  Also used by the
    +     * log Writer
    +     * @param stormRoot the root dist dir for the topology
    +     * @return the classpath for the topology as command line arguments.
    +     * @throws IOException on any error.
    +     */
    +    private List<String> getClassPathParams(final String stormRoot) throws IOException {
    +        final String stormJar = ConfigUtils.supervisorStormJarPath(stormRoot);
    +        final StormTopology stormTopology = ConfigUtils.readSupervisorTopology(_conf, _topologyId, _ops);
    +        final List<String> dependencyLocations = new ArrayList<>();
    +        if (stormTopology.get_dependency_jars() != null) {
    +            for (String dependency : stormTopology.get_dependency_jars()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +
    +        if (stormTopology.get_dependency_artifacts() != null) {
    +            for (String dependency : stormTopology.get_dependency_artifacts()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +        final String workerClassPath = getWorkerClassPath(stormJar, dependencyLocations);
    +        
    +        List<String> classPathParams = new ArrayList<>();
    +        classPathParams.add("-cp");
    +        classPathParams.add(workerClassPath);
    +        return classPathParams;
    +    }
    +    
    +    /**
    +     * Get a set of java properties that are common to both the log writer and the worker processes.
    +     * These are mostly system properties that are used by logging.
    +     * @return a list of command line options
    +     */
    +    private List<String> getCommonParams() {
    +        final String stormHome = ConfigUtils.concatIfNotNull(System.getProperty("storm.home"));
    +        final String workersArtifacts = ConfigUtils.workerArtifactsRoot(_conf);
    +        String stormLogDir = ConfigUtils.getLogDir();
    +        String log4jConfigurationFile = getWorkerLoggingConfigFile(stormHome);
    +        
    +        List<String> commonParams = new ArrayList<>();
    +        commonParams.add("-Dlogfile.name=worker.log");
    +        commonParams.add("-Dstorm.home=" + stormHome);
    +        commonParams.add("-Dworkers.artifacts=" + workersArtifacts);
    +        commonParams.add("-Dstorm.id=" + _topologyId);
    +        commonParams.add("-Dworker.id=" + _workerId);
    +        commonParams.add("-Dworker.port=" + _port);
    +        commonParams.add("-Dstorm.log.dir=" + stormLogDir);
    +        commonParams.add("-Dlog4j.configurationFile=" + log4jConfigurationFile);
    +        commonParams.add("-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector");
    +        return commonParams;
    +    }
    +    
    +    private int getMemOnHeap(WorkerResources resources) {
    +        int memOnheap = 0;
    +        if (resources != null && resources.is_set_mem_on_heap() && 
    +                resources.get_mem_on_heap() > 0) {
    +            memOnheap = (int) Math.ceil(resources.get_mem_on_heap());
    +        } else {
    +            // set the default heap memory size for supervisor-test
    +            memOnheap = Utils.getInt(_topoConf.get(Config.WORKER_HEAP_MEMORY_MB), 768);
    +        }
    +        return memOnheap;
    +    }
    +    
    +    private List<String> getWorkerProfilerChildOpts(int memOnheap) {
    +        List<String> workerProfilerChildopts = new ArrayList<>();
    +        if (Utils.getBoolean(_conf.get(Config.WORKER_PROFILER_ENABLED), false)) {
    +            workerProfilerChildopts = substituteChildopts(_conf.get(Config.WORKER_PROFILER_CHILDOPTS), memOnheap);
    +        }
    +        return workerProfilerChildopts;
    +    }
    +    
    +    /**
    +     * a or b the first one that is not null
    +     * @param a something
    +     * @param b something else
    +     * @return a or b the first one that is not null
    +     */
    +    private <V> V OR(V a, V b) {
    --- End diff --
    
    Should this be in util?


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76303019
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,494 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +    
    +    private class ProcessExitCallback implements Utils.ExitCodeCallable<Void> {
    --- End diff --
    
    I'll look into moving all of the launchProcess etc to the Supervisor package.  That is the only code that uses it anyways. 


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

[GitHub] storm issue #1642: DO NOT MERGE: Please review STORM-2018: Supervisor V2.

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    +1 on general direction. Readability is good; classes aren't too large or convoluted. 


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76309388
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,749 @@
    +/**
    + * 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,
    +        WATING_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 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 monitorFreqMs,
    +                ContainerLauncher containerLauncher, String host, int port,
    +                ISupervisor iSupervisor, LocalState localState) {
    +            this.localizer = localizer;
    +            this.hbTimeoutMs = hbTimeoutMs;
    +            this.firstHbTimeoutMs = firstHbTimeoutMs;
    +            this.containerLauncher = containerLauncher;
    +            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 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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WATING_FOR_WORKER_START) {
    +                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 equivilant(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;
    +                    } else 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 WATING_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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), staticState.port);
    +        }
    +        Time.sleep(1000);
    +        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(1000);
    +        
    +        //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.get_topology_id(), 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);
    +        
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivilant(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WATING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +        
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WATING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    --- End diff --
    
    Too many configs make it very confusing.  I would prefer to just leave it hard coded unless someone has a really valid use case where we would want to make it configurable.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79033354
  
    --- 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);
    +        
    +        //Ignore changes to scheduling while downloading the topology blobs
    +        // We don't support canceling the download through the future yet,
    +        // so to keep everything in sync, just wait
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization, staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        // We don't support canceling the download through the future yet,
    +        // so to keep everything in sync, just wait
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization, staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    --- End diff --
    
    I guess that might clean things up to just go to empty directly.  I have tested this and would prefer to leave it, but if you feel strongly about it I will make the change.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76819266
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,769 @@
    +/**
    + * 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);
    +            }
    +            if (container != null) {
    +                sb.append(" container: ");
    +                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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WAITING_FOR_WORKER_START) {
    +                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 equivilant(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;
    +                    } else 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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), 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.get_topology_id(), 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);
    +        
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivilant(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +        
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for WAITING_FOR_WORKER_START state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleWaitingForWorkerStart(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb != null) {
    +            long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +            if (hbAgeMs <= staticState.hbTimeoutMs) {
    +                return dynamicState.withState(MachineState.RUNNING);
    +            }
    +        }
    +        
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            //We were rescheduled while waiting for the worker to come up
    +            return Slot.killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffms = (Time.currentTimeMillis() - dynamicState.startTime);
    +        if (timeDiffms > staticState.firstHbTimeoutMs) {
    +            LOG.warn("SLOT {}: Container {} failed to launch in {} ms.", staticState.port, dynamicState.container, staticState.firstHbTimeoutMs);
    +            dynamicState.container.kill();
    +            Time.sleep(staticState.killSleepMs);
    +            return dynamicState.withState(MachineState.KILL_AND_RELAUNCH);
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for RUNNING state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleRunning(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            LOG.warn("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAssignment, dynamicState.newAssignment);
    +            //Scheduling changed while running...
    +            return killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        if (dynamicState.container.didMainProcessExit()) {
    +            LOG.warn("SLOT {}: main process has exited", staticState.port);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb == null) {
    +            LOG.warn("SLOT {}: HB returned as null", staticState.port);
    +            //This should never happen, but to be safe
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +        if (timeDiffMs > staticState.hbTimeoutMs) {
    +            LOG.warn("SLOT {}: HB is too old {} > {}", staticState.port, timeDiffMs, staticState.hbTimeoutMs);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        //The worker is up and running check for profiling requests
    +        if (!dynamicState.profileActions.isEmpty()) {
    +            HashSet<TopoProfileAction> mod = new HashSet<>(dynamicState.profileActions);
    +            HashSet<TopoProfileAction> modPending = new HashSet<>(dynamicState.pendingStopProfileActions);
    +            Iterator<TopoProfileAction> iter = mod.iterator();
    +            while (iter.hasNext()) {
    +                TopoProfileAction action = iter.next();
    +                if (!action.topoId.equals(dynamicState.currentAssignment.get_topology_id())) {
    +                    iter.remove();
    +                    LOG.warn("Dropping {} wrong topology is running", action);
    +                    //Not for this topology so skip it
    +                } else {
    +                    if (modPending.contains(action)) {
    +                        boolean isTimeForStop = Time.currentTimeMillis() > action.request.get_time_stamp();
    +                        if (isTimeForStop) {
    +                            if (dynamicState.container.runProfiling(action.request, true)) {
    +                                LOG.debug("Stopped {} action finished", action);
    +                                iter.remove();
    +                                modPending.remove(action);
    +                            } else {
    +                                LOG.warn("Stopping {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            LOG.debug("Still pending {} now: {}", action, Time.currentTimeMillis());
    +                        }
    +                    } else {
    +                        //J_PROFILE_START is not used.  When you see a J_PROFILE_STOP
    +                        // start profiling and save it away to stop when timeout happens
    +                        if (action.request.get_action() == ProfileAction.JPROFILE_STOP) {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                modPending.add(action);
    +                                LOG.debug("Started {} now: {}", action, Time.currentTimeMillis());
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                LOG.debug("Started {} action finished", action);
    +                                iter.remove();
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        }
    +                    }
    +                }
    +            }
    +            dynamicState = dynamicState.withProfileActions(mod, modPending);
    +        }
    +        Time.sleep(staticState.monitorFreqMs);
    +        return dynamicState;
    +    }
    +
    +    static DynamicState handleEmpty(DynamicState dynamicState, StaticState staticState) throws InterruptedException {
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +        }
    +        //Both assignments are null, just wait
    +        if (dynamicState.profileActions != null && !dynamicState.profileActions.isEmpty()) {
    +            //Nothing is scheduled here so throw away all of the profileActions
    +            LOG.warn("Dropping {} no topology is running", dynamicState.profileActions);
    +            dynamicState = dynamicState.withProfileActions(Collections.<TopoProfileAction> emptySet(), Collections.<TopoProfileAction> emptySet());
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +    
    +    private final AtomicReference<LocalAssignment> newAssignment = new AtomicReference<>();
    +    private final AtomicReference<Set<TopoProfileAction>> profiling =
    +            new AtomicReference<Set<TopoProfileAction>>(new HashSet<TopoProfileAction>());
    +    private final StaticState staticState;
    +    private final IStormClusterState clusterState;
    +    private volatile boolean done = false;
    +    private volatile DynamicState dynamicState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignmants;
    +    
    +    public Slot(ILocalizer localizer, Map<String, Object> conf, 
    +            ContainerLauncher containerLauncher, String host,
    +            int port, LocalState localState,
    +            IStormClusterState clusterState,
    +            ISupervisor iSupervisor,
    +            AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignmants) throws Exception {
    +        super("SLOT_"+port);
    +
    +        this.cachedCurrentAssignmants = cachedCurrentAssignmants;
    +        this.clusterState = clusterState;
    +        Map<Integer, LocalAssignment> assignments = localState.getLocalAssignmentsMap();
    +        LocalAssignment currentAssignment = null;
    +        if (assignments != null) {
    +            currentAssignment = assignments.get(port);
    +        }
    +        Container container = null;
    +        if (currentAssignment != null) { 
    +            container = containerLauncher.recoverContainer(port, currentAssignment, localState);
    +        }
    +        
    +        LocalAssignment newAssignment = currentAssignment;
    +        if (currentAssignment != null && container == null) {
    +            currentAssignment = null;
    +            //Assigned something but it is not running
    +        }
    +        
    +        dynamicState = new DynamicState(currentAssignment, container, newAssignment);
    +        staticState = new StaticState(localizer, 
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_START_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_MONITOR_FREQUENCY_SECS)) * 1000,
    +                containerLauncher,
    +                host,
    +                port,
    +                iSupervisor,
    +                localState);
    +        this.newAssignment.set(dynamicState.newAssignment);
    +        if (MachineState.RUNNING == dynamicState.state) {
    +            //We are running so we should recover the blobs.
    +            staticState.localizer.recoverRunningTopology(currentAssignment.get_topology_id(), port);
    +        }
    +        LOG.warn("SLOT {}:{} Starting in state {} - assignment {}", staticState.host, staticState.port, dynamicState.state, dynamicState.currentAssignment);
    +    }
    +    
    +    public MachineState getMachineState() {
    +        return dynamicState.state;
    +    }
    +    
    +    /**
    +     * Set a new assignment asynchronously
    +     * @param newAssignment the new assignment for this slot to run, null to run nothing
    +     */
    +    public void setNewAssignment(LocalAssignment newAssignment) {
    +        this.newAssignment.set(newAssignment);
    +    }
    +    
    +    public void addProfilerActions(Set<TopoProfileAction> actions) {
    +        if (actions != null) {
    +            while(true) {
    +                Set<TopoProfileAction> orig = profiling.get();
    +                Set<TopoProfileAction> newActions = new HashSet<>(orig);
    +                newActions.addAll(actions);
    +                if (profiling.compareAndSet(orig, newActions)) {
    +                    return;
    +                }
    +            }
    +        }
    +    }
    +    
    +    public void run() {
    +        try {
    +            while(!done) {
    +                LocalAssignment localNewAssignment = newAssignment.get();
    +                Set<TopoProfileAction> origProfileActions = new HashSet<>(profiling.get());
    +                Set<TopoProfileAction> removed = new HashSet<>(origProfileActions);
    +                
    +                DynamicState nextState = 
    +                        stateMachineStep(dynamicState.withNewAssignment(localNewAssignment)
    +                                .withProfileActions(origProfileActions, dynamicState.pendingStopProfileActions), staticState);
    +
    +                if (LOG.isDebugEnabled() || dynamicState.state != nextState.state) {
    +                    LOG.info("STATE {} -> {}", dynamicState, nextState);
    +                }
    +                //Save the current state for recovery
    +                if (!equivilant(nextState.currentAssignment, dynamicState.currentAssignment)) {
    +                    LOG.warn("SLOT {}: Changing current assignment from {} to {}", staticState.port, dynamicState.currentAssignment, nextState.currentAssignment);
    +                    synchronized(staticState.localState) {
    +                        Map<Integer, LocalAssignment> assignments = staticState.localState.getLocalAssignmentsMap();
    +                        if (assignments == null) {
    +                            assignments = new HashMap<>();
    +                        }
    +                        if (nextState.currentAssignment == null) {
    +                            assignments.remove(staticState.port);
    +                        } else {
    +                            assignments.put(staticState.port, nextState.currentAssignment);
    +                        }
    +                        staticState.localState.setLocalAssignmentsMap(assignments);
    +                    }
    +                    Map<Long, LocalAssignment> update = null;
    +                    Map<Long, LocalAssignment> orig = null;
    +                    do {
    +                        Long lport = new Long(staticState.port);
    +                        orig = cachedCurrentAssignmants.get();
    +                        update = new HashMap<>(orig);
    +                        if (nextState.currentAssignment == null) {
    +                            update.remove(lport);
    +                        } else {
    +                            update.put(lport, nextState.currentAssignment);
    +                        }
    +                    } while (!cachedCurrentAssignmants.compareAndSet(orig, update));
    +                }
    +                
    +                // clean up the profiler actions that are not being processed
    +                removed.removeAll(dynamicState.profileActions);
    +                removed.removeAll(dynamicState.pendingStopProfileActions);
    +                for (TopoProfileAction action: removed) {
    +                    try {
    +                        clusterState.deleteTopologyProfileRequests(action.topoId, action.request);
    +                    } catch (Exception e) {
    +                        LOG.error("Error trying to remove profiling request, it will be retried", e);
    +                    }
    +                }
    +                Set<TopoProfileAction> orig, copy;
    +                do {
    +                    orig = profiling.get();
    +                    copy = new HashSet<>(orig);
    +                    copy.removeAll(removed);
    +                } while (!profiling.compareAndSet(orig, copy));
    +                dynamicState = nextState;
    +                newAssignment.compareAndSet(localNewAssignment, dynamicState.newAssignment);
    --- End diff --
    
    why is this compareAndSet? Also, in what case shouldn't newAssignment's current value not be == localNewAssignment? Maybe what I am missing is a comment.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79041849
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,422 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.net.URLDecoder;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + */
    +public class AsyncLocalizer implements ILocalizer, Shutdownable {
    +    /**
    +     * A future that has already completed.
    +     */
    +    private static class AllDoneFuture implements Future<Void> {
    +
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isCancelled() {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isDone() {
    +            return true;
    +        }
    +
    +        @Override
    +        public Void get() {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void get(long timeout, TimeUnit unit) {
    +            return null;
    +        }
    +
    +    }
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizer.class);
    +
    +    private final Localizer _localizer;
    +    private final ExecutorService _execService;
    +    private final boolean _isLocalMode;
    +    private final Map<String, Object> _conf;
    +    private final Map<String, LocalDownloadedResource> _basicPending;
    +    private final Map<String, LocalDownloadedResource> _blobPending;
    +    private final AdvancedFSOps _fsOps;
    +
    +    private class DownloadBaseBlobsDistributed implements Callable<Void> {
    +        protected final String _topologyId;
    +        protected final File _stormRoot;
    +        
    +        public DownloadBaseBlobsDistributed(String topologyId) throws IOException {
    +            _topologyId = topologyId;
    +            _stormRoot = new File(ConfigUtils.supervisorStormDistRoot(_conf, _topologyId));
    +        }
    +        
    +        protected void downloadBaseBlobs(File tmproot) throws Exception {
    +            String stormJarKey = ConfigUtils.masterStormJarKey(_topologyId);
    +            String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +            String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +            String jarPath = ConfigUtils.supervisorStormJarPath(tmproot.getAbsolutePath());
    +            String codePath = ConfigUtils.supervisorStormCodePath(tmproot.getAbsolutePath());
    +            String confPath = ConfigUtils.supervisorStormConfPath(tmproot.getAbsolutePath());
    +            _fsOps.forceMkdir(tmproot);
    +            _fsOps.restrictDirectoryPermissions(tmproot);
    +            ClientBlobStore blobStore = Utils.getClientBlobStoreForSupervisor(_conf);
    +            try {
    +                Utils.downloadResourcesAsSupervisor(stormJarKey, jarPath, blobStore);
    +                Utils.downloadResourcesAsSupervisor(stormCodeKey, codePath, blobStore);
    +                Utils.downloadResourcesAsSupervisor(stormConfKey, confPath, blobStore);
    +            } finally {
    +                blobStore.shutdown();
    +            }
    +            Utils.extractDirFromJar(jarPath, ConfigUtils.RESOURCES_SUBDIR, tmproot);
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            if (_fsOps.fileExists(_stormRoot)) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    _fsOps.deleteIfExists(_stormRoot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            File tr = new File(tmproot);
    +            try {
    +                downloadBaseBlobs(tr);
    +                _fsOps.moveDirectoryPreferAtomic(tr, _stormRoot);
    +                _fsOps.setupStormCodeDir(ConfigUtils.readSupervisorStormConf(_conf, _topologyId), _stormRoot);
    +                deleteAll = false;
    +            } finally {
    +                if (deleteAll) {
    +                    LOG.warn("Failed to download basic resources for topology-id {}", _topologyId);
    +                    _fsOps.deleteIfExists(tr);
    +                    _fsOps.deleteIfExists(_stormRoot);
    +                }
    +            }
    +            return null;
    +        }
    +    }
    +    
    +    private class DownloadBaseBlobsLocal extends DownloadBaseBlobsDistributed {
    +
    +        public DownloadBaseBlobsLocal(String topologyId) throws IOException {
    +            super(topologyId);
    +        }
    +        
    +        @Override
    +        protected void downloadBaseBlobs(File tmproot) throws Exception {
    +            _fsOps.forceMkdir(tmproot);
    +            String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +            String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +            File codePath = new File(ConfigUtils.supervisorStormCodePath(tmproot.getAbsolutePath()));
    +            File confPath = new File(ConfigUtils.supervisorStormConfPath(tmproot.getAbsolutePath()));
    +            BlobStore blobStore = Utils.getNimbusBlobStore(_conf, null);
    +            try {
    +                try (OutputStream codeOutStream = _fsOps.getOutputStream(codePath)){
    +                    blobStore.readBlobTo(stormCodeKey, codeOutStream, null);
    +                }
    +                try (OutputStream confOutStream = _fsOps.getOutputStream(confPath)) {
    +                    blobStore.readBlobTo(stormConfKey, confOutStream, null);
    +                }
    +            } finally {
    +                blobStore.shutdown();
    +            }
    +
    +            ClassLoader classloader = Thread.currentThread().getContextClassLoader();
    +            String resourcesJar = AsyncLocalizer.resourcesJar();
    +            URL url = classloader.getResource(ConfigUtils.RESOURCES_SUBDIR);
    +
    +            String targetDir = tmproot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +
    +            if (resourcesJar != null) {
    +                LOG.info("Extracting resources from jar at {} to {}", resourcesJar, targetDir);
    +                Utils.extractDirFromJar(resourcesJar, ConfigUtils.RESOURCES_SUBDIR, _stormRoot);
    +            } else if (url != null) {
    +                LOG.info("Copying resources at {} to {} ", url.toString(), targetDir);
    +                if ("jar".equals(url.getProtocol())) {
    +                    JarURLConnection urlConnection = (JarURLConnection) url.openConnection();
    +                    Utils.extractDirFromJar(urlConnection.getJarFileURL().getFile(), ConfigUtils.RESOURCES_SUBDIR, _stormRoot);
    +                } else {
    +                    _fsOps.copyDirectory(new File(url.getFile()), new File(targetDir));
    +                }
    +            }
    +        }
    +    }
    +    
    +    private class DownloadBlobs implements Callable<Void> {
    +        private final String _topologyId;
    +
    +        public DownloadBlobs(String topologyId) {
    +            _topologyId = topologyId;
    +        }
    +
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    --- End diff --
    
    You're right, it should be fine. I've mostly been using ExecutorService in a context where we didn't call .get (f. ex. scheduling), so I may be a little trigger happy :)


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79040636
  
    --- 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);
    +        
    --- End diff --
    
    That is the thing I really don't like putting anything into an assert with side effects, because it will behave differently between testing and production.  In this case it really should not matter, because the side effect is minimal, and it is just timing.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @harshach did you get a chance to take a look yet?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79042371
  
    --- 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);
    +        
    --- End diff --
    
    Makes sense. Feel free to leave it out. 


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    I recovered it and fixed some issues with integration tests/rat.  I think it should be good to go.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by abellina <gi...@git.apache.org>.
Github user abellina commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76818353
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Slot.java ---
    @@ -0,0 +1,769 @@
    +/**
    + * 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);
    +            }
    +            if (container != null) {
    +                sb.append(" container: ");
    +                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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WAITING_FOR_WORKER_START) {
    +                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 equivilant(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;
    +                    } else 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
    +     */
    +    static DynamicState prepareForNewAssignmentOnEmptySlot(DynamicState dynamicState, StaticState staticState) {
    +        assert(dynamicState.container == null);
    +        
    +        if (dynamicState.newAssignment == null) {
    +            return dynamicState.withState(MachineState.EMPTY);
    +        }
    +        Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment.get_topology_id(), 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.get_topology_id(), 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.get_topology_id(), 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);
    +        
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivilant(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization.get_topology_id(), staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +        
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    +            throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds");
    +        }
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for WAITING_FOR_WORKER_START state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleWaitingForWorkerStart(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb != null) {
    +            long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +            if (hbAgeMs <= staticState.hbTimeoutMs) {
    +                return dynamicState.withState(MachineState.RUNNING);
    +            }
    +        }
    +        
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            //We were rescheduled while waiting for the worker to come up
    +            return Slot.killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffms = (Time.currentTimeMillis() - dynamicState.startTime);
    +        if (timeDiffms > staticState.firstHbTimeoutMs) {
    +            LOG.warn("SLOT {}: Container {} failed to launch in {} ms.", staticState.port, dynamicState.container, staticState.firstHbTimeoutMs);
    +            dynamicState.container.kill();
    +            Time.sleep(staticState.killSleepMs);
    +            return dynamicState.withState(MachineState.KILL_AND_RELAUNCH);
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for RUNNING state.
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleRunning(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            LOG.warn("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAssignment, dynamicState.newAssignment);
    +            //Scheduling changed while running...
    +            return killContainerForChangedAssignment(dynamicState, staticState);
    +        }
    +        if (dynamicState.container.didMainProcessExit()) {
    +            LOG.warn("SLOT {}: main process has exited", staticState.port);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        LSWorkerHeartbeat hb = dynamicState.container.readHeartbeat();
    +        if (hb == null) {
    +            LOG.warn("SLOT {}: HB returned as null", staticState.port);
    +            //This should never happen, but to be safe
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        long timeDiffMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000;
    +        if (timeDiffMs > staticState.hbTimeoutMs) {
    +            LOG.warn("SLOT {}: HB is too old {} > {}", staticState.port, timeDiffMs, staticState.hbTimeoutMs);
    +            return killAndRelaunchContainer(dynamicState, staticState);
    +        }
    +        
    +        //The worker is up and running check for profiling requests
    +        if (!dynamicState.profileActions.isEmpty()) {
    +            HashSet<TopoProfileAction> mod = new HashSet<>(dynamicState.profileActions);
    +            HashSet<TopoProfileAction> modPending = new HashSet<>(dynamicState.pendingStopProfileActions);
    +            Iterator<TopoProfileAction> iter = mod.iterator();
    +            while (iter.hasNext()) {
    +                TopoProfileAction action = iter.next();
    +                if (!action.topoId.equals(dynamicState.currentAssignment.get_topology_id())) {
    +                    iter.remove();
    +                    LOG.warn("Dropping {} wrong topology is running", action);
    +                    //Not for this topology so skip it
    +                } else {
    +                    if (modPending.contains(action)) {
    +                        boolean isTimeForStop = Time.currentTimeMillis() > action.request.get_time_stamp();
    +                        if (isTimeForStop) {
    +                            if (dynamicState.container.runProfiling(action.request, true)) {
    +                                LOG.debug("Stopped {} action finished", action);
    +                                iter.remove();
    +                                modPending.remove(action);
    +                            } else {
    +                                LOG.warn("Stopping {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            LOG.debug("Still pending {} now: {}", action, Time.currentTimeMillis());
    +                        }
    +                    } else {
    +                        //J_PROFILE_START is not used.  When you see a J_PROFILE_STOP
    +                        // start profiling and save it away to stop when timeout happens
    +                        if (action.request.get_action() == ProfileAction.JPROFILE_STOP) {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                modPending.add(action);
    +                                LOG.debug("Started {} now: {}", action, Time.currentTimeMillis());
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        } else {
    +                            if (dynamicState.container.runProfiling(action.request, false)) {
    +                                LOG.debug("Started {} action finished", action);
    +                                iter.remove();
    +                            } else {
    +                                LOG.warn("Starting {} failed, will be retried", action);
    +                            }
    +                        }
    +                    }
    +                }
    +            }
    +            dynamicState = dynamicState.withProfileActions(mod, modPending);
    +        }
    +        Time.sleep(staticState.monitorFreqMs);
    +        return dynamicState;
    +    }
    +
    +    static DynamicState handleEmpty(DynamicState dynamicState, StaticState staticState) throws InterruptedException {
    +        if (!equivilant(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +            return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +        }
    +        //Both assignments are null, just wait
    +        if (dynamicState.profileActions != null && !dynamicState.profileActions.isEmpty()) {
    +            //Nothing is scheduled here so throw away all of the profileActions
    +            LOG.warn("Dropping {} no topology is running", dynamicState.profileActions);
    +            dynamicState = dynamicState.withProfileActions(Collections.<TopoProfileAction> emptySet(), Collections.<TopoProfileAction> emptySet());
    +        }
    +        Time.sleep(1000);
    +        return dynamicState;
    +    }
    +    
    +    private final AtomicReference<LocalAssignment> newAssignment = new AtomicReference<>();
    +    private final AtomicReference<Set<TopoProfileAction>> profiling =
    +            new AtomicReference<Set<TopoProfileAction>>(new HashSet<TopoProfileAction>());
    +    private final StaticState staticState;
    +    private final IStormClusterState clusterState;
    +    private volatile boolean done = false;
    +    private volatile DynamicState dynamicState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignmants;
    +    
    +    public Slot(ILocalizer localizer, Map<String, Object> conf, 
    +            ContainerLauncher containerLauncher, String host,
    +            int port, LocalState localState,
    +            IStormClusterState clusterState,
    +            ISupervisor iSupervisor,
    +            AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignmants) throws Exception {
    +        super("SLOT_"+port);
    +
    +        this.cachedCurrentAssignmants = cachedCurrentAssignmants;
    +        this.clusterState = clusterState;
    +        Map<Integer, LocalAssignment> assignments = localState.getLocalAssignmentsMap();
    +        LocalAssignment currentAssignment = null;
    +        if (assignments != null) {
    +            currentAssignment = assignments.get(port);
    +        }
    +        Container container = null;
    +        if (currentAssignment != null) { 
    +            container = containerLauncher.recoverContainer(port, currentAssignment, localState);
    +        }
    +        
    +        LocalAssignment newAssignment = currentAssignment;
    +        if (currentAssignment != null && container == null) {
    +            currentAssignment = null;
    +            //Assigned something but it is not running
    +        }
    +        
    +        dynamicState = new DynamicState(currentAssignment, container, newAssignment);
    +        staticState = new StaticState(localizer, 
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_START_TIMEOUT_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS)) * 1000,
    +                Utils.getInt(conf.get(Config.SUPERVISOR_MONITOR_FREQUENCY_SECS)) * 1000,
    +                containerLauncher,
    +                host,
    +                port,
    +                iSupervisor,
    +                localState);
    +        this.newAssignment.set(dynamicState.newAssignment);
    --- End diff --
    
    should call setNewAssignment?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78974501
  
    --- 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);
    +        
    +        //Ignore changes to scheduling while downloading the topology blobs
    +        // We don't support canceling the download through the future yet,
    +        // so to keep everything in sync, just wait
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization, staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    +        
    +        //Ignore changes to scheduling while downloading the topology code
    +        // We don't support canceling the download through the future yet,
    +        // so to keep everything in sync, just wait
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization, staticState.port);
    +            return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION);
    +        } catch (TimeoutException e) {
    +            return dynamicState;
    +        }
    +    }
    +
    +    /**
    +     * State Transitions for KILL state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            LOG.warn("SLOT {} all processes are dead...", staticState.port);
    +            return cleanupCurrentContainer(dynamicState, staticState, 
    +                    dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION);
    +        }
    +
    +        LOG.warn("SLOT {} force kill and wait...", staticState.port);
    +        dynamicState.container.forceKill();
    +        Time.sleep(staticState.killSleepMs);
    +        return dynamicState;
    +    }
    +
    +    /**
    +     * State Transitions for KILL_AND_RELAUNCH state.
    +     * PRECONDITION: container.kill() was called
    +     * PRECONDITION: container != null && currentAssignment != null
    +     * @param dynamicState current state
    +     * @param staticState static data
    +     * @return the next state
    +     * @throws Exception on any error
    +     */
    +    static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.container != null);
    +        assert(dynamicState.currentAssignment != null);
    +        
    +        if (dynamicState.container.areAllProcessesDead()) {
    +            if (equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) {
    +                dynamicState.container.cleanUpForRestart();
    +                dynamicState.container.relaunch();
    +                return dynamicState.withState(MachineState.WAITING_FOR_WORKER_START);
    +            }
    +            //Scheduling changed after we killed all of the processes
    +            return prepareForNewAssignmentOnEmptySlot(cleanupCurrentContainer(dynamicState, staticState, null), staticState);
    +        }
    +        //The child processes typically exit in < 1 sec.  If 2 mins later they are still around something is wrong
    +        if ((Time.currentTimeMillis() - dynamicState.startTime) > 120000) {
    --- End diff --
    
    Nitpick: Consider writing the two minutes at 120_000 to make it easy to read at a glance


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77418692
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,420 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + * TODO once we have replaced the original supervisor merge this with
    + * Localizer and optimize them
    + */
    +public class AsyncLocalizer implements ILocalizer, Shutdownable {
    +    /**
    +     * A future that has already completed.
    +     */
    +    private static class AllDoneFuture implements Future<Void> {
    +
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isCancelled() {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isDone() {
    +            return true;
    +        }
    +
    +        @Override
    +        public Void get() {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void get(long timeout, TimeUnit unit) {
    +            return null;
    +        }
    +
    +    }
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizer.class);
    +
    +    private final Localizer _localizer;
    +    private final ExecutorService _execService;
    +    private final boolean _isLocalMode;
    +    private final Map<String, Object> _conf;
    +    private final Map<String, LocalDownloadedResource> _basicPending;
    +    private final Map<String, LocalDownloadedResource> _blobPending;
    +    private final AdvancedFSOps _fsOps;
    +
    +    private class DownloadBaseBlobsDistributed implements Callable<Void> {
    +        private final String _topologyId;
    +        
    +        public DownloadBaseBlobsDistributed(String topologyId) {
    +            this._topologyId = topologyId;
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            File sr = new File(stormroot);
    +            if (sr.exists()) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    Utils.forceDelete(stormroot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            try {
    +                String stormJarKey = ConfigUtils.masterStormJarKey(_topologyId);
    +                String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +                String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +                String jarPath = ConfigUtils.supervisorStormJarPath(tmproot);
    +                String codePath = ConfigUtils.supervisorStormCodePath(tmproot);
    +                String confPath = ConfigUtils.supervisorStormConfPath(tmproot);
    +                FileUtils.forceMkdir(new File(tmproot));
    +                _fsOps.restrictDirectoryPermissions(tmproot);
    +                ClientBlobStore blobStore = Utils.getClientBlobStoreForSupervisor(_conf);
    +                try {
    +                    Utils.downloadResourcesAsSupervisor(stormJarKey, jarPath, blobStore);
    +                    Utils.downloadResourcesAsSupervisor(stormCodeKey, codePath, blobStore);
    +                    Utils.downloadResourcesAsSupervisor(stormConfKey, confPath, blobStore);
    +                } finally {
    +                    blobStore.shutdown();
    +                }
    +                Utils.extractDirFromJar(jarPath, ConfigUtils.RESOURCES_SUBDIR, tmproot);
    +                _fsOps.moveDirectoryPreferAtomic(new File(tmproot), new File(stormroot));
    +                SupervisorUtils.setupStormCodeDir(_conf, ConfigUtils.readSupervisorStormConf(_conf, _topologyId), stormroot);
    +                deleteAll = false;
    +            } finally {
    +                if (deleteAll) {
    +                    LOG.info("Failed to download basic resources for topology-id {}", _topologyId);
    +                    Utils.forceDelete(tmproot);
    +                    Utils.forceDelete(stormroot);
    +                }
    +            }
    +            return null;
    +        }
    +    }
    +    
    +    private class DownloadBaseBlobsLocal implements Callable<Void> {
    +        private final String _topologyId;
    +        
    +        public DownloadBaseBlobsLocal(String topologyId) {
    +            this._topologyId = topologyId;
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            File sr = new File(stormroot);
    +            if (sr.exists()) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    Utils.forceDelete(stormroot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            try {
    +                BlobStore blobStore = Utils.getNimbusBlobStore(_conf, null, null);
    +                FileOutputStream codeOutStream = null;
    +                FileOutputStream confOutStream = null;
    +                try {
    +                    FileUtils.forceMkdir(new File(tmproot));
    +                    String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +                    String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +                    String codePath = ConfigUtils.supervisorStormCodePath(tmproot);
    +                    String confPath = ConfigUtils.supervisorStormConfPath(tmproot);
    +                    codeOutStream = new FileOutputStream(codePath);
    +                    blobStore.readBlobTo(stormCodeKey, codeOutStream, null);
    +                    confOutStream = new FileOutputStream(confPath);
    +                    blobStore.readBlobTo(stormConfKey, confOutStream, null);
    +                } finally {
    +                    if (codeOutStream != null)
    +                        codeOutStream.close();
    +                    if (confOutStream != null)
    +                        codeOutStream.close();
    +                    blobStore.shutdown();
    +                }
    +
    +                ClassLoader classloader = Thread.currentThread().getContextClassLoader();
    +                String resourcesJar = AsyncLocalizer.resourcesJar();
    +                URL url = classloader.getResource(ConfigUtils.RESOURCES_SUBDIR);
    +
    +                String targetDir = tmproot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +
    +                if (resourcesJar != null) {
    +                    LOG.info("Extracting resources from jar at {} to {}", resourcesJar, targetDir);
    +                    Utils.extractDirFromJar(resourcesJar, ConfigUtils.RESOURCES_SUBDIR, stormroot);
    +                } else if (url != null) {
    +                    LOG.info("Copying resources at {} to {} ", url.toString(), targetDir);
    +                    if (url.getProtocol() == "jar") {
    +                        JarURLConnection urlConnection = (JarURLConnection) url.openConnection();
    +                        Utils.extractDirFromJar(urlConnection.getJarFileURL().getFile(), ConfigUtils.RESOURCES_SUBDIR, stormroot);
    +                    } else {
    +                        FileUtils.copyDirectory(new File(url.getFile()), (new File(targetDir)));
    +                    }
    +                }
    +                _fsOps.moveDirectoryPreferAtomic(new File(tmproot), new File(stormroot));
    +                SupervisorUtils.setupStormCodeDir(_conf, ConfigUtils.readSupervisorStormConf(_conf, _topologyId), stormroot);
    +                deleteAll = false;
    +            } finally {
    +                if (deleteAll) {
    +                    LOG.info("Failed to download basic resources for topology-id {}", _topologyId);
    +                    Utils.forceDelete(tmproot);
    +                    Utils.forceDelete(stormroot);
    --- End diff --
    
    IOException


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78961556
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,328 @@
    +/**
    + * 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.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 java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +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.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this.superConf = supervisor.getConf();
    +        this.stormClusterState = supervisor.getStormClusterState();
    +        this.syncSupEventManager = supervisor.getEventManger();
    +        this.assignmentVersions = new AtomicReference<>(new HashMap<>());
    +        this.assignmentId = supervisor.getAssignmentId();
    +        this.iSuper = supervisor.getiSupervisor();
    +        this.localizer = supervisor.getAsyncLocalizer();
    +        this.host = supervisor.getHostName();
    +        this.localState = supervisor.getLocalState();
    +        this.clusterState = supervisor.getStormClusterState();
    +        this.cachedAssignments = supervisor.getCurrAssignment();
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, supervisor.getSharedContext());
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +        
    +        try {
    +            Collection<String> workers = SupervisorUtils.supervisorWorkerIds(superConf);
    +            for (Slot slot: slots.values()) {
    +                String workerId = slot.getWorkerId();
    +                if (workerId != null) {
    +                    workers.remove(workerId);
    +                }
    +            }
    +            if (!workers.isEmpty()) {
    +                supervisor.killWorkers(workers, launcher);
    +            }
    +        } catch (Exception e) {
    +            LOG.warn("Error trying to clean up old workers", e);
    +        }
    +
    +        //All the slots/assignments should be recovered now, so we can clean up anything that we don't expect to be here
    +        try {
    +            localizer.cleanupUnusedTopologies();
    +        } catch (Exception e) {
    +            LOG.warn("Error trying to clean up old topologies", e);
    +        }
    +        
    +        for (Slot slot: slots.values()) {
    +            slot.start();
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        return new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    +                            if (actions == null) {
    +                                actions = new HashSet<>();
    +                                filtered.put(port.intValue(), actions);
    +                            }
    +                            actions.add(new TopoProfileAction(topoId, req));
    +                        }
    +                    }
    +                }
    +            }
    +            
    +            for (Integer port: allPorts) {
    +                Slot slot = slots.get(port);
    +                if (slot == null) {
    +                    slot = mkSlot(port);
    +                    slots.put(port, slot);
    +                    slot.start();
    +                }
    +                slot.setNewAssignment(allAssignments.get(port));
    +                slot.addProfilerActions(filtered.get(port));
    +            }
    +            
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +    }
    +    
    +    protected Map<String, VersionedData<Assignment>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +            Map<String, VersionedData<Assignment>> localAssignmentVersion, Runnable callback) throws Exception {
    +        Map<String, VersionedData<Assignment>> updateAssignmentVersion = new HashMap<>();
    +        for (String topoId : topoIds) {
    +            Integer recordedVersion = -1;
    +            Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +            VersionedData<Assignment> locAssignment = localAssignmentVersion.get(topoId);
    +            if (locAssignment != null) {
    +                recordedVersion = locAssignment.getVersion();
    +            }
    +            if (version == null) {
    +                // ignore
    +            } else if (version == recordedVersion) {
    +                updateAssignmentVersion.put(topoId, locAssignment);
    +            } else {
    +                VersionedData<Assignment> assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                updateAssignmentVersion.put(topoId, assignmentVersion);
    +            }
    +        }
    +        return updateAssignmentVersion;
    +    }
    +    
    +    protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +        Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +        for (String stormId : stormIds) {
    +            List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +            ret.put(stormId, profileRequests);
    +        }
    +        return ret;
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readAssignments(Map<String, VersionedData<Assignment>> assignmentsSnapshot,
    +            String assignmentId, AtomicInteger retries) {
    +        try {
    +            Map<Integer, LocalAssignment> portLA = new HashMap<Integer, LocalAssignment>();
    --- End diff --
    
    Nitpick: Could leave off \<Integer, LocalAssignment\> on the right hand side.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    With my patch (symlink issue and NPE issue) I can see workers launched and killed by Supervisor V2. (remote)
    
    Tested:
    - kill worker process with -9
    - rebalance with different worker count
    - jstack dump, heap dump, restart worker


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    The Travis failures are from hive dependencies not being downloaded.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77414257
  
    --- Diff: storm-core/test/jvm/org/apache/storm/daemon/supervisor/BasicContainerTest.java ---
    @@ -0,0 +1,459 @@
    +package org.apache.storm.daemon.supervisor;
    +
    +import static org.junit.Assert.*;
    +import static org.mockito.Mockito.*;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.junit.Test;
    +
    +public class BasicContainerTest {
    +    public static class CommandRun {
    +        final List<String> cmd;
    +        final Map<String, String> env;
    +        final File pwd;
    +        
    +        public CommandRun(List<String> cmd, Map<String, String> env, File pwd) {
    +            this.cmd = cmd;
    +            this.env = env;
    +            this.pwd = pwd;
    +        }
    +    }
    +    
    +    public static class MockBasicContainer extends BasicContainer {
    +        public final List<CommandRun> profileCmds = new ArrayList<>();
    +        public final List<CommandRun> workerCmds = new ArrayList<>();
    +        
    +        public MockBasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf,
    +                String supervisorId, LocalState localState, ResourceIsolationInterface resourceIsolationManager,
    +                boolean recover) throws IOException {
    +            super(port, assignment, conf, supervisorId, localState, resourceIsolationManager, recover);
    +        }
    +        
    +        public MockBasicContainer(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +                Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +                ResourceIsolationInterface resourceIsolationManager, LocalState localState,
    +                String profileCmd) throws IOException {
    +            super(ops, port, assignment, conf, topoConf, supervisorId, resourceIsolationManager, localState, profileCmd);
    +        }
    +        
    +        @Override
    +        protected Map<String, Object> readTopoConf() throws IOException {
    +            return new HashMap<>();
    +        }
    +        
    +        @Override
    +        public void createNewWorkerId() {
    +            super.createNewWorkerId();
    +        }
    +        
    +        @Override
    +        public List<String> substituteChildopts(Object value, int memOnheap) {
    +            return super.substituteChildopts(value, memOnheap);
    +        }
    +               
    +        @Override
    +        protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +                File targetDir) throws IOException, InterruptedException {
    +            profileCmds.add(new CommandRun(command, env, targetDir));
    +            return true;
    +        }
    +        
    +        @Override
    +        protected void launchWorkerProcess(List<String> command, Map<String, String> env, String logPrefix,
    +                ExitCodeCallback processExitCallback, File targetDir) throws IOException {
    +            workerCmds.add(new CommandRun(command, env, targetDir));
    +        }
    +        
    +        @Override
    +        protected String javaCmd(String cmd) {
    +            //avoid system dependent things
    +            return cmd;
    +        }
    +        
    +        @Override
    +        protected List<String> frameworkClasspath() {
    +            //We are not really running anything so make this
    +            // simple to check for
    +            return Arrays.asList("FRAMEWORK_CP");
    +        }
    +        
    +        @Override
    +        protected String javaLibraryPath(String stormRoot, Map<String, Object> conf) {
    +            return "JLP";
    +        }
    +    }
    +    
    +    @Test
    +    public void testCreateNewWorkerId() throws Exception {
    +        final String topoId = "test_topology";
    +        final int port = 8080;
    +        LocalAssignment la = new LocalAssignment();
    +        la.set_topology_id(topoId);
    +        
    +        AdvancedFSOps ops = mock(AdvancedFSOps.class);
    +        
    +        LocalState ls = mock(LocalState.class);
    +        
    +        MockBasicContainer mc = new MockBasicContainer(ops, port, la, new HashMap<String, Object>(), 
    +                new HashMap<String, Object>(), "SUPERVISOR", null, ls, "profile");
    +        
    +        mc.createNewWorkerId();
    +        
    +        assertNotNull(mc._workerId);
    +        verify(ls).getApprovedWorkers();
    +        Map<String, Integer> expectedNewState = new HashMap<String, Integer>();
    +        expectedNewState.put(mc._workerId, port);
    +        verify(ls).setApprovedWorkers(expectedNewState);
    +    }
    +    
    +    @Test
    +    public void testRecovery() throws Exception {
    +        final String topoId = "test_topology";
    +        final String workerId = "myWorker";
    +        final int port = 8080;
    +        LocalAssignment la = new LocalAssignment();
    +        la.set_topology_id(topoId);
    +        
    +        Map<String, Integer> workerState = new HashMap<String, Integer>();
    +        workerState.put(workerId, port);
    +        
    +        LocalState ls = mock(LocalState.class);
    +        when(ls.getApprovedWorkers()).thenReturn(workerState);
    +        
    +        MockBasicContainer mc = new MockBasicContainer(port, la, new HashMap<String, Object>(), 
    +                "SUPERVISOR", ls, null, true);
    +        
    +        assertEquals(workerId, mc._workerId);
    +    }
    +    
    +    @Test
    +    public void testRecoveryMiss() throws Exception {
    +        final String topoId = "test_topology";
    +        final int port = 8080;
    +        LocalAssignment la = new LocalAssignment();
    +        la.set_topology_id(topoId);
    +        
    +        Map<String, Integer> workerState = new HashMap<String, Integer>();
    +        workerState.put("somethingelse", port+1);
    +        
    +        LocalState ls = mock(LocalState.class);
    +        when(ls.getApprovedWorkers()).thenReturn(workerState);
    +        
    +        try {
    +            new MockBasicContainer(port, la, new HashMap<String, Object>(), 
    +                    "SUPERVISOR", ls, null, true);
    +            fail("Container recovered worker incorrectly");
    +        } catch (ContainerRecoveryException e) {
    +            //Expected
    +        }
    +    }
    +    
    +    @Test
    +    public void testCleanUp() throws Exception {
    +        final String topoId = "test_topology";
    +        final int port = 8080;
    +        final String workerId = "worker-id";
    +        LocalAssignment la = new LocalAssignment();
    +        la.set_topology_id(topoId);
    +        
    +        AdvancedFSOps ops = mock(AdvancedFSOps.class);
    +        
    +        Map<String, Integer> workerState = new HashMap<String, Integer>();
    +        workerState.put(workerId, port);
    +        
    +        LocalState ls = mock(LocalState.class);
    +        when(ls.getApprovedWorkers()).thenReturn(new HashMap<>(workerState));
    +        
    +        MockBasicContainer mc = new MockBasicContainer(ops, port, la, new HashMap<String, Object>(), 
    +                new HashMap<String, Object>(), "SUPERVISOR", null, ls, "profile");
    +        mc._workerId = workerId;
    +        
    +        mc.cleanUp();
    +        
    +        assertNull(mc._workerId);
    +        verify(ls).getApprovedWorkers();
    +        Map<String, Integer> expectedNewState = new HashMap<String, Integer>();
    +        verify(ls).setApprovedWorkers(expectedNewState);
    +    }
    +    
    +    @Test
    +    public void testRunProfiling() throws Exception {
    +        final long pid = 100;
    +        final String topoId = "test_topology";
    +        final int port = 8080;
    +        final String workerId = "worker-id";
    +        final String stormLocal = ContainerTest.asAbsPath("tmp", "testing");
    +        final String topoRoot = ContainerTest.asAbsPath(stormLocal, topoId, String.valueOf(port));
    +        final File workerArtifactsPid = ContainerTest.asAbsFile(topoRoot, "worker.pid");
    +        
    +        final Map<String, Object> superConf = new HashMap<>();
    +        superConf.put(Config.STORM_LOCAL_DIR, stormLocal);
    +        superConf.put(Config.STORM_WORKERS_ARTIFACTS_DIR, stormLocal);
    +        
    +        LocalAssignment la = new LocalAssignment();
    +        la.set_topology_id(topoId);
    +        
    +        AdvancedFSOps ops = mock(AdvancedFSOps.class);
    +        when(ops.slurpString(workerArtifactsPid)).thenReturn(String.valueOf(pid));
    +        
    +        LocalState ls = mock(LocalState.class);
    +        
    +        MockBasicContainer mc = new MockBasicContainer(ops, port, la, superConf, 
    +                new HashMap<String, Object>(), "SUPERVISOR", null, ls, "profile");
    +        mc._workerId = workerId;
    +        
    +        //HEAP DUMP
    +        ProfileRequest req = new ProfileRequest();
    +        req.set_action(ProfileAction.JMAP_DUMP);
    +        
    +        mc.runProfiling(req, false);
    +        
    +        assertEquals(1, mc.profileCmds.size());
    +        CommandRun cmd = mc.profileCmds.get(0);
    +        mc.profileCmds.clear();
    +        assertEquals(Arrays.asList("profile", String.valueOf(pid), "jmap", topoRoot), cmd.cmd);
    +        assertEquals(new File(topoRoot), cmd.pwd);
    +        
    +        //JSTACK DUMP
    +        req.set_action(ProfileAction.JSTACK_DUMP);
    +        
    +        mc.runProfiling(req, false);
    +        
    +        assertEquals(1, mc.profileCmds.size());
    +        cmd = mc.profileCmds.get(0);
    +        mc.profileCmds.clear();
    +        assertEquals(Arrays.asList("profile", String.valueOf(pid), "jstack", topoRoot), cmd.cmd);
    +        assertEquals(new File(topoRoot), cmd.pwd);
    +        
    +        //RESTART
    +        req.set_action(ProfileAction.JVM_RESTART);
    +        
    +        mc.runProfiling(req, false);
    +        
    +        assertEquals(1, mc.profileCmds.size());
    +        cmd = mc.profileCmds.get(0);
    +        mc.profileCmds.clear();
    +        assertEquals(Arrays.asList("profile", String.valueOf(pid), "kill"), cmd.cmd);
    +        assertEquals(new File(topoRoot), cmd.pwd);
    +        
    +        //JPROFILE DUMP
    +        req.set_action(ProfileAction.JPROFILE_DUMP);
    +        
    +        mc.runProfiling(req, false);
    +        
    +        assertEquals(1, mc.profileCmds.size());
    +        cmd = mc.profileCmds.get(0);
    +        mc.profileCmds.clear();
    +        assertEquals(Arrays.asList("profile", String.valueOf(pid), "dump", topoRoot), cmd.cmd);
    +        assertEquals(new File(topoRoot), cmd.pwd);
    +        
    +        //JPROFILE START
    +        req.set_action(ProfileAction.JPROFILE_STOP);
    --- End diff --
    
    Ya it was really confusing until I asked @kishorvpatil about it and he explained it all to me.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77151542
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +
    +    //For testing purposes
    +    public BasicContainer(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager, LocalState localState,
    +            String profileCmd) throws IOException {
    +        super(ops, port, assignment, conf, topoConf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +        _profileCmd = profileCmd;
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState localState, ResourceIsolationInterface resourceIsolationManager, boolean recover)
    +            throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                LOG.info("Recovered Worker {}", wid);
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException(
    +                    "Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            removeWorkersOn(workerToPort, _port);
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +            LOG.info("Created Worker ID {}", _workerId);
    +        }
    +    }
    +
    +    private static void removeWorkersOn(Map<String, Integer> workerToPort, int _port) {
    +        for (Iterator<Entry<String, Integer>> i = workerToPort.entrySet().iterator(); i.hasNext();) {
    +            Entry<String, Integer> found = i.next();
    +            if (_port == found.getValue().intValue()) {
    +                i.remove();
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUpForRestart() throws IOException {
    +        super.cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            removeWorkersOn(workersToPort, _port);
    +            _localState.setApprovedWorkers(workersToPort);
    +            LOG.info("Removed Worker ID {}", _workerId);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        setup();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = _ops.slurpString(new File(str)).trim();
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        if (command.size() > 0) {
    +            return runProfilingCommand(command, env, logPrefix, targetFile);
    +        }
    +        LOG.warn("PROFILING REQUEST NOT SUPPORTED {} IGNORED...", request);
    +        return true;
    +    }
    +
    +    /**
    +     * Get the command to run when doing profiling
    +     * @param action the profiling action to perform
    +     * @param stop if this is meant to stop the profiling or start it
    +     * @param workerPid the PID of the process to profile
    +     * @param targetDir the current working directory of the worker process
    +     * @return the command to run for profiling.
    +     */
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        switch(action) {
    +            case JMAP_DUMP:
    +                return jmapDumpCmd(workerPid, targetDir);
    +            case JSTACK_DUMP:
    +                return jstackDumpCmd(workerPid, targetDir);
    +            case JPROFILE_DUMP:
    +                return jprofileDump(workerPid, targetDir);
    +            case JVM_RESTART:
    +                return jprofileJvmRestart(workerPid);
    +            case JPROFILE_STOP:
    +                if (stop) {
    +                    return jprofileStop(workerPid, targetDir);
    +                }
    +                return jprofileStart(workerPid);
    +            default:
    +                return Lists.newArrayList();
    +        }
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +
    +    /**
    +     * Compute the java.library.path that should be used for the worker.
    +     * This helps it to load JNI libraries that are packaged in the uber jar.
    +     * @param stormRoot the root directory of the worker process
    +     * @param conf the config for the supervisor.
    +     * @return the java.library.path/LD_LIBRARY_PATH to use so native libraries load correctly.
    +     */
    +    protected String jlp(String stormRoot, Map<String, Object> conf) {
    +        String resourceRoot = stormRoot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +        String os = System.getProperty("os.name").replaceAll("\\s+", "_");
    +        String arch = System.getProperty("os.arch");
    +        String archResourceRoot = resourceRoot + Utils.FILE_PATH_SEPARATOR + os + "-" + arch;
    +        String ret = CPJ.join(archResourceRoot, resourceRoot,
    +                conf.get(Config.JAVA_LIBRARY_PATH));
    +        return ret;
    +    }
    +
    +    /**
    +     * Returns a collection of jar file names found under the given directory.
    +     * @param dir the directory to search
    +     * @return the jar file names
    +     */
    +    protected List<String> getFullJars(File dir) {
    +        File[] files = dir.listFiles(jarFilter);
    +
    +        if (files == null) {
    +            return Collections.emptyList();
    +        }
    +
    +        return Arrays.stream(files).map(f -> f.getAbsolutePath())
    +                .collect(Collectors.toList());
    +    }
    +    
    +    protected List<String> frameworkClasspath() {
    +        String stormHome = System.getProperty("storm.home");
    +
    +        File stormLibDir = new File(stormHome, "lib");
    +        String stormConfDir =
    +                System.getenv("STORM_CONF_DIR") != null ?
    +                System.getenv("STORM_CONF_DIR") :
    +                new File(stormHome, "conf").getAbsolutePath();
    +        File stormExtlibDir = new File(stormHome, "extlib");
    +        String extcp = System.getenv("STORM_EXT_CLASSPATH");
    +        List<String> pathElements = new LinkedList<>();
    +        pathElements.addAll(getFullJars(stormLibDir));
    +        pathElements.addAll(getFullJars(stormExtlibDir));
    +        pathElements.add(extcp);
    +        pathElements.add(stormConfDir);
    +
    +        return pathElements;
    +    }
    +    
    +    @SuppressWarnings("unchecked")
    +    private List<String> asStringList(Object o) {
    +        if (o instanceof String) {
    +            return Arrays.asList((String)o);
    +        } else if (o instanceof List) {
    +            return (List<String>)o;
    +        }
    +        return Collections.EMPTY_LIST;
    +    }
    +    
    +    /**
    +     * Compute the classpath for the worker process
    +     * @param stormJar the topology jar
    +     * @param dependencyLocations any dependencies from the topology
    +     * @return the full classpath
    +     */
    +    protected String getWorkerClassPath(String stormJar, List<String> dependencyLocations) {
    +        List<String> workercp = new ArrayList<>();
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH_BEGINNING)));
    +        workercp.addAll(frameworkClasspath());
    +        workercp.add(stormJar);
    +        workercp.addAll(dependencyLocations);
    +        workercp.addAll(asStringList(_topoConf.get(Config.TOPOLOGY_CLASSPATH)));
    +        return CPJ.join(workercp);
    +    }
    +
    +    private String substituteChildOptsInternal(String string, int memOnheap) {
    +        if (StringUtils.isNotBlank(string)) {
    +            String p = String.valueOf(_port);
    +            string = string.replace("%ID%", p);
    +            string = string.replace("%WORKER-ID%", _workerId);
    +            string = string.replace("%TOPOLOGY-ID%", _topologyId);
    +            string = string.replace("%WORKER-PORT%", p);
    +            if (memOnheap > 0) {
    +                string = string.replace("%HEAP-MEM%", String.valueOf(memOnheap));
    +            }
    +        }
    +        return string;
    +    }
    +    
    +    protected List<String> substituteChildopts(Object value) {
    +        return substituteChildopts(value, -1);
    +    }
    +
    +    protected List<String> substituteChildopts(Object value, int memOnheap) {
    +        List<String> rets = new ArrayList<>();
    +        if (value instanceof String) {
    +            String string = substituteChildOptsInternal((String) value, memOnheap);
    +            if (StringUtils.isNotBlank(string)) {
    +                String[] strings = string.split("\\s+");
    +                rets.addAll(Arrays.asList(strings));
    +            }
    +        } else if (value instanceof List) {
    +            @SuppressWarnings("unchecked")
    +            List<String> objects = (List<String>) value;
    +            for (String object : objects) {
    +                String str = substituteChildOptsInternal(object, memOnheap);
    +                if (StringUtils.isNotBlank(str)) {
    +                    rets.add(str);
    +                }
    +            }
    +        }
    +        return rets;
    +    }
    +
    +    /**
    +     * Launch the worker process (non-blocking)
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param processExitcallback
    +     *            a callback for when the process exits
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     */
    +    protected void launchWorkerProcess(List<String> command, Map<String, String> env, String logPrefix,
    +            ExitCodeCallback processExitCallback, File targetDir) throws IOException {
    +        SupervisorUtils.launchProcess(command, env, logPrefix, processExitCallback, targetDir);
    +    }
    +
    +    private String getWorkerLoggingConfigFile(String stormHome) {
    +        String log4jConfigurationDir = (String) (_conf.get(Config.STORM_LOG4J2_CONF_DIR));
    +
    +        if (StringUtils.isNotBlank(log4jConfigurationDir)) {
    +            if (!Utils.isAbsolutePath(log4jConfigurationDir)) {
    +                log4jConfigurationDir = stormHome + Utils.FILE_PATH_SEPARATOR + log4jConfigurationDir;
    +            }
    +        } else {
    +            log4jConfigurationDir = stormHome + Utils.FILE_PATH_SEPARATOR + "log4j2";
    +        }
    + 
    +        if (Utils.IS_ON_WINDOWS && !log4jConfigurationDir.startsWith("file:")) {
    +            log4jConfigurationDir = "file:///" + log4jConfigurationDir;
    +        }
    +        return log4jConfigurationDir + Utils.FILE_PATH_SEPARATOR + "worker.xml";
    +    }
    +    
    +    /**
    +     * Get parameters for the class path of the worker process.  Also used by the
    +     * log Writer
    +     * @param stormRoot the root dist dir for the topology
    +     * @return the classpath for the topology as command line arguments.
    +     * @throws IOException on any error.
    +     */
    +    private List<String> getClassPathParams(final String stormRoot) throws IOException {
    +        final String stormJar = ConfigUtils.supervisorStormJarPath(stormRoot);
    +        final StormTopology stormTopology = ConfigUtils.readSupervisorTopology(_conf, _topologyId, _ops);
    +        final List<String> dependencyLocations = new ArrayList<>();
    +        if (stormTopology.get_dependency_jars() != null) {
    +            for (String dependency : stormTopology.get_dependency_jars()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +
    +        if (stormTopology.get_dependency_artifacts() != null) {
    +            for (String dependency : stormTopology.get_dependency_artifacts()) {
    +                dependencyLocations.add(new File(stormRoot, dependency).getAbsolutePath());
    +            }
    +        }
    +        final String workerClassPath = getWorkerClassPath(stormJar, dependencyLocations);
    +        
    +        List<String> classPathParams = new ArrayList<>();
    +        classPathParams.add("-cp");
    +        classPathParams.add(workerClassPath);
    +        return classPathParams;
    +    }
    +    
    +    /**
    +     * Get a set of java properties that are common to both the log writer and the worker processes.
    +     * These are mostly system properties that are used by logging.
    +     * @return a list of command line options
    +     */
    +    private List<String> getCommonParams() {
    +        final String stormHome = ConfigUtils.concatIfNotNull(System.getProperty("storm.home"));
    +        final String workersArtifacts = ConfigUtils.workerArtifactsRoot(_conf);
    +        String stormLogDir = ConfigUtils.getLogDir();
    +        String log4jConfigurationFile = getWorkerLoggingConfigFile(stormHome);
    +        
    +        List<String> commonParams = new ArrayList<>();
    +        commonParams.add("-Dlogfile.name=worker.log");
    +        commonParams.add("-Dstorm.home=" + stormHome);
    --- End diff --
    
    Could we put some of these properties in constants somewhere?


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76476827
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,417 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.AdvancedFSOps;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill the processes in this container nicely.
    +     * kill -15 equivalent
    +     * @throws IOException on any error
    +     */
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Kill the processes in this container violently.
    +     * kill -9 equivalent
    +     * @throws IOException on any error
    +     */
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq"+pid);
    +            pb.redirectError(Redirect.INHERIT);
    +            Process p = pb.start();
    +            try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +                //TODO would be really nice to parse out the user
    +                if (in.readLine() != null) {
    +                    return true;
    +                }
    +            }
    +            return false;
    +        }
    +        //Not on Windows assume POSIX
    +        ProcessBuilder pb = new ProcessBuilder("ps", "-o", "user", "-p", String.valueOf(pid));
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            String first = in.readLine();
    +            assert("USER".equals(first));
    +            String processUser;
    +            while ((processUser = in.readLine()) != null) {
    +                if (user.equals(processUser)) {
    +                    return true;
    +                } else {
    +                    LOG.info("Found {} running as {}, but expected it to be {}", pid, processUser, user);
    +                }
    +            }
    +        }
    +        return false;
    +    }
    +    
    +    /**
    +     * @return true if all of the processes are dead, else false
    +     * @throws IOException on any error
    +     */
    +    public boolean areAllProcessesDead() throws IOException {
    +        Set<Long> pids = getAllPids();
    +        String user = getWorkerUser();
    +        
    +        boolean allDead = true;
    +        for (Long pid: pids) {
    +            if (!isProcessAlive(pid, user)) {
    +                LOG.warn("{}: PID {} is dead", _workerId, pid);
    +            } else {
    +                allDead = false;
    +                break;
    +            }
    +        }
    +        return allDead;
    +    }
    +
    +    /**
    +     * Clean up the container. It is not coming back.
    +     * by default do the same thing as when restarting.
    +     * @throws IOException on any error
    +     */
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +    }
    +
    +    /**
    +     * Setup the container to run.  By default this creates the needed directories/links in the
    +     * local file system
    +     * PREREQUISITE: All needed blobs and topology, jars/configs have been downloaded and
    +     * placed in the appropriate locations
    +     * @throws IOException on any error
    +     */
    +    public void setup() throws IOException {
    +        final String topologyId = _assignment.get_topology_id();
    +        if (!SupervisorUtils.doRequiredTopoFilesExist(_conf, topologyId)) {
    +            LOG.info("Missing topology storm code, so can't launch  worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                    _supervisorId, _port, _workerId);
    +            throw new IllegalStateException("Not all needed files are here!!!!");
    +        }
    +        String pidsPath = ConfigUtils.workerPidsRoot(_conf, _workerId);
    +        String hbPath = ConfigUtils.workerHeartbeatsRoot(_conf, _workerId);
    +    
    +        FileUtils.forceMkdir(new File(pidsPath));
    +        FileUtils.forceMkdir(new File(ConfigUtils.workerTmpRoot(_conf, _workerId)));
    +        FileUtils.forceMkdir(new File(hbPath));
    +    
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        String user = (String) topologyConf.get(Config.TOPOLOGY_SUBMITTER_USER);
    +        writeLogMetadata(topologyConf, user, topologyId);
    +        ConfigUtils.setWorkerUserWSE(_conf, _workerId, user);
    +        createArtifactsLink(topologyId);
    +    
    +        createBlobstoreLinks(topologyId);
    +    }
    +    
    +    /**
    +     * Write out the file used by the log viewer to allow/reject log access
    +     * @param topologyConf the config for the topology
    +     * @param user the user this is going to run as
    +     * @param topologyId the id of the topology
    +     * @throws IOException on any error
    +     */
    +    @SuppressWarnings("unchecked")
    +    protected void writeLogMetadata(Map<String, Object> topologyConf, String user, String topologyId) throws IOException {
    +        Map<String, Object> data = new HashMap<>();
    +        data.put(Config.TOPOLOGY_SUBMITTER_USER, user);
    +        data.put("worker-id", _workerId);
    +
    +        Set<String> logsGroups = new HashSet<>();
    +        //for supervisor-test
    +        if (topologyConf.get(Config.LOGS_GROUPS) != null) {
    +            List<String> groups = (List<String>) topologyConf.get(Config.LOGS_GROUPS);
    +            for (String group : groups){
    +                logsGroups.add(group);
    +            }
    +        }
    +        if (topologyConf.get(Config.TOPOLOGY_GROUPS) != null) {
    +            List<String> topGroups = (List<String>) topologyConf.get(Config.TOPOLOGY_GROUPS);
    +            logsGroups.addAll(topGroups);
    +        }
    +        data.put(Config.LOGS_GROUPS, logsGroups.toArray());
    +
    +        Set<String> logsUsers = new HashSet<>();
    +        if (topologyConf.get(Config.LOGS_USERS) != null) {
    +            List<String> logUsers = (List<String>) topologyConf.get(Config.LOGS_USERS);
    +            for (String logUser : logUsers){
    +                logsUsers.add(logUser);
    +            }
    +        }
    +        if (topologyConf.get(Config.TOPOLOGY_USERS) != null) {
    +            List<String> topUsers = (List<String>) topologyConf.get(Config.TOPOLOGY_USERS);
    +            for (String logUser : topUsers){
    +                logsUsers.add(logUser);
    +            }
    +        }
    +        data.put(Config.LOGS_USERS, logsUsers.toArray());
    +
    +        File file = ConfigUtils.getLogMetaDataFile(_conf, topologyId, _port);
    +        File parent = file.getParentFile();
    +        if (!Utils.checkFileExists(parent)) {
    +            FileUtils.forceMkdir(file.getParentFile());
    +            _ops.setupStormCodeDir(ConfigUtils.readSupervisorStormConf(_conf, topologyId), file.getParentFile().getCanonicalPath());
    +        }
    +        Yaml yaml = new Yaml();
    +        try (FileWriter writer = new FileWriter(file)) {
    +            yaml.dump(data, writer);
    +        }
    +    }
    +    
    +    /**
    +     * Create symlink from the containers directory/artifacts to the artifacts directory
    +     * @param topologyId the id of the topology this is for
    +     * @throws IOException on any error
    +     */
    +    protected void createArtifactsLink(String topologyId) throws IOException {
    +        String workerDir = ConfigUtils.workerRoot(_conf, _workerId);
    +        String topoDir = ConfigUtils.workerArtifactsRoot(_conf, topologyId);
    +        if (Utils.checkFileExists(workerDir)) {
    +            LOG.debug("Creating symlinks for worker-id: {} topology-id: {} to its port artifacts directory", _workerId, topologyId);
    +            Utils.createSymlink(workerDir, topoDir, "artifacts", String.valueOf(_port));
    +        }
    --- End diff --
    
    In `writeLogMetadata` we actually try to set up the directory if we don't find it first. Then we write the log metadata there. Is this a similar case when if we find that the dir is not there already we should throw?


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    Found other issue: 
    Constructor of BasicContainer (in fact constructor of Container) throws FNF when topology dist files are deleted. So even though Slot is creating with recover mode, Supervisor is killed instead of throwing ContainerRecoveryException. Supervisor will consistently be killed unless we remove the assignment for that - killing topology.
    
    This is a weird case but can be happening, and I guess that's why ContainerRecoveryException exists.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76637608
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,300 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.Writer;
    +import java.nio.file.FileSystems;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.Paths;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    +    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
    +    
    +    /**
    +     * Factory to create a new AdvancedFSOps
    +     * @param conf the configuration of the process
    +     * @return the appropriate instance of the class for this config and environment.
    +     */
    +    public static AdvancedFSOps mk(Map<String, Object> conf) {
    +        if (Utils.isOnWindows()) {
    +            return new AdvancedWindowsFSOps(conf);
    +        }
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new AdvancedRunAsUserFSOps(conf);
    +        }
    +        return new AdvancedFSOps();
    +    }
    +    
    +    private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
    +        private final Map<String, Object> _conf;
    +        
    +        public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
    +            if (Utils.isOnWindows()) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +            _conf = conf;
    +        }
    +        
    +        @Override
    +        public void setupBlobPermissions(File path, String user) throws IOException {
    +            String logPrefix = "setup blob permissions for " + path;
    +            SupervisorUtils.processLauncherAndWait(_conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
    +        }
    +        
    +        @Override
    +        public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +            String absolutePath = path.getAbsolutePath();
    +            LOG.debug("Deleting path {}", absolutePath);
    +            if (user == null) {
    +                user = Files.getOwner(path.toPath()).getName();
    +            }
    +            List<String> commands = new ArrayList<>();
    +            commands.add("rmr");
    +            commands.add(absolutePath);
    +            SupervisorUtils.processLauncherAndWait(_conf, user, commands, null, logPrefix);
    +            if (Utils.checkFileExists(absolutePath)) {
    +                throw new RuntimeException(path + " was not deleted.");
    +            }
    +        }
    +        
    +        @Override
    +        public void setupStormCodeDir(Map<String, Object> topologyConf, File path) throws IOException {
    +            SupervisorUtils.setupStormCodeDir(_conf, topologyConf, path.getCanonicalPath());
    +        }
    +    }
    +    
    +    /**
    +     * Operations that need to override the default ones when running on Windows
    +     *
    +     */
    +    private static class AdvancedWindowsFSOps extends AdvancedFSOps {
    +
    +        public AdvancedWindowsFSOps(Map<String, Object> conf) {
    +            if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +        }
    +        
    +        @Override
    +        public void restrictDirectoryPermissions(String dir) throws IOException {
    +            //NOOP, if windows gets support for run as user we will need to find a way to suppor this
    +        }
    +        
    +        @Override
    +        public void moveDriectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // This is not atomic but it does work
    +            FileUtils.moveDirectory(fromDir, toDir);
    +        }
    +        
    +        @Override
    +        public boolean supportsAtomicDirectoryMove() {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // FileUtils.moveDirectory is not atomic
    +            return false;
    +        }
    +    }
    +    
    +    
    +    protected AdvancedFSOps() {
    +        //NOOP, but restricted permissions
    +    }
    +
    +    /**
    +     * Set directory permissions to (OWNER)RWX (GROUP)R-X (OTHER)---
    +     * On some systems that do not support this, it may become a noop
    +     * @param dir the directory to change permissions on
    +     * @throws IOException on any error
    +     */
    +    public void restrictDirectoryPermissions(String dir) throws IOException {
    +        Set<PosixFilePermission> perms = new HashSet<>(
    +                Arrays.asList(PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE,
    +                        PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ,
    +                        PosixFilePermission.GROUP_EXECUTE));
    +        Files.setPosixFilePermissions(FileSystems.getDefault().getPath(dir), perms);
    +    }
    +
    +    /**
    +     * Move fromDir to toDir, and try to make it an atomic move if possible
    +     * @param fromDir what to move
    +     * @param toDir where to move it from
    +     * @throws IOException on any error
    +     */
    +    public void moveDriectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +        FileUtils.forceMkdir(toDir);
    +        Files.move(fromDir.toPath(), toDir.toPath(), StandardCopyOption.ATOMIC_MOVE);
    +    }
    +    
    +    /**
    +     * @return true if an atomic directory move works, else false.
    +     */
    +    public boolean supportsAtomicDirectoryMove() {
    +        return true;
    +    }
    +    
    +    /**
    +     * Setup permissions properly for an internal blob store path
    +     * @param path the path to the permissions
    +     * @param user the user to change the permissions for
    +     * @throws IOException on any error
    +     */
    +    public void setupBlobPermissions(File path, String user) throws IOException {
    +        //Normally this is a NOOP
    +    }
    +
    +    /**
    +     * Delete a file or a directory and all of the children. If it exists.
    +     * @param path what to delete
    +     * @param user who to delete it as if doing it as someone else is supported
    +     * @param logPrefix if an external process needs to be launched to delete 
    +     * the object what prefix to include in the logs
    +     * @throws IOException on any error.
    +     */
    +    public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +        LOG.debug("Deleting path {}", path);
    --- End diff --
    
    Again, can this be info level log entry?


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    @revans2 Thanks, looks great. +1 for merging.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76997296
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,319 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.Writer;
    +import java.nio.file.FileSystems;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    +    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
    +    
    +    /**
    +     * Factory to create a new AdvancedFSOps
    +     * @param conf the configuration of the process
    +     * @return the appropriate instance of the class for this config and environment.
    +     */
    +    public static AdvancedFSOps make(Map<String, Object> conf) {
    +        if (Utils.isOnWindows()) {
    +            return new AdvancedWindowsFSOps(conf);
    +        }
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new AdvancedRunAsUserFSOps(conf);
    +        }
    +        return new AdvancedFSOps();
    +    }
    +    
    +    private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
    +        private final Map<String, Object> _conf;
    +        
    +        public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
    +            if (Utils.isOnWindows()) {
    +                throw new UnsupportedOperationException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +            _conf = conf;
    +        }
    +        
    +        @Override
    +        public void setupBlobPermissions(File path, String user) throws IOException {
    +            String logPrefix = "setup blob permissions for " + path;
    +            SupervisorUtils.processLauncherAndWait(_conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
    +        }
    +        
    +        @Override
    +        public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +            String absolutePath = path.getAbsolutePath();
    +            LOG.info("Deleting path {}", absolutePath);
    +            if (user == null) {
    +                user = Files.getOwner(path.toPath()).getName();
    +            }
    +            List<String> commands = new ArrayList<>();
    +            commands.add("rmr");
    +            commands.add(absolutePath);
    +            SupervisorUtils.processLauncherAndWait(_conf, user, commands, null, logPrefix);
    +            if (Utils.checkFileExists(absolutePath)) {
    +                throw new RuntimeException(path + " was not deleted.");
    +            }
    +        }
    +        
    +        @Override
    +        public void setupStormCodeDir(Map<String, Object> topologyConf, File path) throws IOException {
    +            SupervisorUtils.setupStormCodeDir(_conf, topologyConf, path.getCanonicalPath());
    +        }
    +    }
    +    
    +    /**
    +     * Operations that need to override the default ones when running on Windows
    +     *
    +     */
    +    private static class AdvancedWindowsFSOps extends AdvancedFSOps {
    +
    +        public AdvancedWindowsFSOps(Map<String, Object> conf) {
    +            if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +        }
    +        
    +        @Override
    +        public void restrictDirectoryPermissions(String dir) throws IOException {
    +            //NOOP, if windows gets support for run as user we will need to find a way to suppor this
    +        }
    +        
    +        @Override
    +        public void moveDriectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // This is not atomic but it does work
    +            FileUtils.moveDirectory(fromDir, toDir);
    +        }
    +        
    +        @Override
    +        public boolean supportsAtomicDirectoryMove() {
    +            // Files/move with non-empty directory doesn't work well on Windows
    +            // FileUtils.moveDirectory is not atomic
    +            return false;
    +        }
    +    }
    +    
    +    
    +    protected AdvancedFSOps() {
    +        //NOOP, but restricted permissions
    +    }
    +
    +    /**
    +     * Set directory permissions to (OWNER)RWX (GROUP)R-X (OTHER)---
    +     * On some systems that do not support this, it may become a noop
    +     * @param dir the directory to change permissions on
    +     * @throws IOException on any error
    +     */
    +    public void restrictDirectoryPermissions(String dir) throws IOException {
    +        Set<PosixFilePermission> perms = new HashSet<>(
    +                Arrays.asList(PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE,
    +                        PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ,
    +                        PosixFilePermission.GROUP_EXECUTE));
    +        Files.setPosixFilePermissions(FileSystems.getDefault().getPath(dir), perms);
    +    }
    +
    +    /**
    +     * Move fromDir to toDir, and try to make it an atomic move if possible
    +     * @param fromDir what to move
    +     * @param toDir where to move it from
    +     * @throws IOException on any error
    +     */
    +    public void moveDriectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    +        FileUtils.forceMkdir(toDir);
    +        Files.move(fromDir.toPath(), toDir.toPath(), StandardCopyOption.ATOMIC_MOVE);
    +    }
    +    
    +    /**
    +     * @return true if an atomic directory move works, else false.
    +     */
    +    public boolean supportsAtomicDirectoryMove() {
    +        return true;
    +    }
    +    
    +    /**
    +     * Setup permissions properly for an internal blob store path
    +     * @param path the path to set the permissions on
    +     * @param user the user to change the permissions for
    +     * @throws IOException on any error
    +     */
    +    public void setupBlobPermissions(File path, String user) throws IOException {
    +        //Normally this is a NOOP
    +    }
    +
    +    /**
    +     * Delete a file or a directory and all of the children. If it exists.
    +     * @param path what to delete
    +     * @param user who to delete it as if doing it as someone else is supported
    +     * @param logPrefix if an external process needs to be launched to delete 
    +     * the object what prefix to include in the logs
    +     * @throws IOException on any error.
    +     */
    +    public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +        LOG.info("Deleting path {}", path);
    --- End diff --
    
    Great catch.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76637435
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,313 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.Writer;
    +import java.nio.file.FileSystems;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    +    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
    +    
    +    /**
    +     * Factory to create a new AdvancedFSOps
    +     * @param conf the configuration of the process
    +     * @return the appropriate instance of the class for this config and environment.
    +     */
    +    public static AdvancedFSOps make(Map<String, Object> conf) {
    +        if (Utils.isOnWindows()) {
    +            return new AdvancedWindowsFSOps(conf);
    +        }
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new AdvancedRunAsUserFSOps(conf);
    +        }
    +        return new AdvancedFSOps();
    +    }
    +    
    +    private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
    +        private final Map<String, Object> _conf;
    +        
    +        public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
    +            if (Utils.isOnWindows()) {
    +                throw new UnsupportedOperationException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +            _conf = conf;
    +        }
    +        
    +        @Override
    +        public void setupBlobPermissions(File path, String user) throws IOException {
    +            String logPrefix = "setup blob permissions for " + path;
    +            SupervisorUtils.processLauncherAndWait(_conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
    +        }
    +        
    +        @Override
    +        public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +            String absolutePath = path.getAbsolutePath();
    +            LOG.debug("Deleting path {}", absolutePath);
    --- End diff --
    
    This can probably be info level.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79029515
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected final String _stormHome = System.getProperty("storm.home");
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +    
    +    /**
    +     * Create a new BasicContainer
    +     * @param type the type of container being made.
    +     * @param conf the supervisor config
    +     * @param supervisorId the ID of the supervisor this is a part of.
    +     * @param port the port the container is on.  Should be <= 0 if only a partial recovery
    +     * @param assignment the assignment for this container. Should be null if only a partial recovery.
    +     * @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used.
    +     * @param localState the local state of the supervisor.  May be null if partial recovery
    +     * @param workerId the id of the worker to use.  Must not be null if doing a partial recovery.
    +     * @param ops file system operations (mostly for testing) if null a new one is made
    +     * @param topoConf the config of the topology (mostly for testing) if null 
    +     * and not a partial recovery the real conf is read.
    +     * @param profileCmd the command to use when profiling (used for testing)
    +     */
    +    public BasicContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int port,
    +            LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
    +            LocalState localState, String workerId, Map<String, Object> topoConf, 
    +            AdvancedFSOps ops, String profileCmd) throws IOException {
    +        super(type, conf, supervisorId, port, assignment, resourceIsolationManager, workerId, topoConf, ops);
    +        assert(localState != null);
    +        _localState = localState;
    +
    +        if (type.isRecovery() && !type.isOnlyKillable()) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    --- End diff --
    
    The exception is actually OK, it will result in the slot being Empty which is logged already.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77360881
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    +                            if (actions == null) {
    +                                actions = new HashSet<>();
    +                                filtered.put(port.intValue(), actions);
    +                            }
    +                            actions.add(new TopoProfileAction(topoId, req));
    +                        }
    +                    }
    +                }
    +            }
    +            
    +            for (Integer port: allPorts) {
    +                Slot slot = slots.get(port);
    +                if (slot == null) {
    +                    slot = mkSlot(port);
    +                    slots.put(port, slot);
    +                }
    +                slot.setNewAssignment(allAssignments.get(port));
    +                slot.addProfilerActions(filtered.get(port));
    +            }
    +            
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +    }
    +    
    +    protected Map<String, VersionedData<Assignment>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +            Map<String, VersionedData<Assignment>> localAssignmentVersion, Runnable callback) throws Exception {
    +        Map<String, VersionedData<Assignment>> updateAssignmentVersion = new HashMap<>();
    +        for (String topoId : topoIds) {
    +            Integer recordedVersion = -1;
    +            Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +            VersionedData<Assignment> locAssignment = localAssignmentVersion.get(topoId);
    +            if (locAssignment != null) {
    +                recordedVersion = locAssignment.getVersion();
    +            }
    +            if (version == null) {
    +                // ignore
    +            } else if (version == recordedVersion) {
    +                updateAssignmentVersion.put(topoId, locAssignment);
    +            } else {
    +                VersionedData<Assignment> assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                updateAssignmentVersion.put(topoId, assignmentVersion);
    +            }
    +        }
    +        return updateAssignmentVersion;
    +    }
    +    
    +    protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +        Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +        for (String stormId : stormIds) {
    +            List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +            ret.put(stormId, profileRequests);
    +        }
    +        return ret;
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readAssignments(Map<String, VersionedData<Assignment>> assignmentsSnapshot,
    +            String assignmentId, AtomicInteger retries) {
    +        try {
    +            Map<Integer, LocalAssignment> portLA = new HashMap<Integer, LocalAssignment>();
    +            for (Map.Entry<String, VersionedData<Assignment>> assignEntry : assignmentsSnapshot.entrySet()) {
    +                String topoId = assignEntry.getKey();
    +                Assignment assignment = assignEntry.getValue().getData();
    +
    +                Map<Integer, LocalAssignment> portTasks = readMyExecutors(topoId, assignmentId, assignment);
    +
    +                for (Map.Entry<Integer, LocalAssignment> entry : portTasks.entrySet()) {
    +
    +                    Integer port = entry.getKey();
    +
    +                    LocalAssignment la = entry.getValue();
    +
    +                    if (!portLA.containsKey(port)) {
    +                        portLA.put(port, la);
    +                    } else {
    +                        throw new RuntimeException("Should not have multiple topologies assigned to one port");
    +                    }
    +                }
    +            }
    +            retries.set(0);
    +            return portLA;
    +        } catch (RuntimeException e) {
    +            if (retries.get() > 2) {
    +                throw e;
    +            } else {
    +                retries.addAndGet(1);
    +            }
    +            LOG.warn("{} : retrying {} of 3", e.getMessage(), retries.get());
    +            return null;
    +        }
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readMyExecutors(String stormId, String assignmentId, Assignment assignment) {
    +        Map<Integer, LocalAssignment> portTasks = new HashMap<>();
    +        Map<Long, WorkerResources> slotsResources = new HashMap<>();
    +        Map<NodeInfo, WorkerResources> nodeInfoWorkerResourcesMap = assignment.get_worker_resources();
    +        if (nodeInfoWorkerResourcesMap != null) {
    +            for (Map.Entry<NodeInfo, WorkerResources> entry : nodeInfoWorkerResourcesMap.entrySet()) {
    +                if (entry.getKey().get_node().equals(assignmentId)) {
    +                    Set<Long> ports = entry.getKey().get_port();
    +                    for (Long port : ports) {
    +                        slotsResources.put(port, entry.getValue());
    +                    }
    +                }
    +            }
    +        }
    +        Map<List<Long>, NodeInfo> executorNodePort = assignment.get_executor_node_port();
    +        if (executorNodePort != null) {
    +            for (Map.Entry<List<Long>, NodeInfo> entry : executorNodePort.entrySet()) {
    +                if (entry.getValue().get_node().equals(assignmentId)) {
    +                    for (Long port : entry.getValue().get_port()) {
    +                        LocalAssignment localAssignment = portTasks.get(port.intValue());
    +                        if (localAssignment == null) {
    +                            List<ExecutorInfo> executors = new ArrayList<ExecutorInfo>();
    +                            localAssignment = new LocalAssignment(stormId, executors);
    +                            if (slotsResources.containsKey(port)) {
    +                                localAssignment.set_resources(slotsResources.get(port));
    +                            }
    +                            portTasks.put(port.intValue(), localAssignment);
    +                        }
    +                        List<ExecutorInfo> executorInfoList = localAssignment.get_executors();
    +                        executorInfoList.add(new ExecutorInfo(entry.getKey().get(0).intValue(), entry.getKey().get(entry.getKey().size() - 1).intValue()));
    +                    }
    +                }
    +            }
    +        }
    +        return portTasks;
    +    }
    +
    +    public synchronized void shutdownAllWorkers() {
    +        for (Slot slot: slots.values()) {
    +            try {
    +                slot.setNewAssignment(null);
    +            } catch (Exception e) {
    +                LOG.error("Error trying to shutdown workers in {}", slot, e);
    +            }
    --- End diff --
    
    Are there Exceptions here we need to catch here?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78971801
  
    --- 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 --
    
    Consider asserting that state is empty


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77369714
  
    --- 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);
    +            }
    +            if (container != null) {
    +                sb.append(" container: ");
    +                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 new DynamicState(this.state, this.newAssignment,
    +                    this.container, this.currentAssignment,
    +                    this.pendingLocalization, this.startTime,
    +                    pendingDownload, this.profileActions,
    +                    this.pendingStopProfileActions);
    +        }
    +        
    +        public DynamicState withState(final MachineState state) {
    +            long newStartTime = this.startTime;
    +            if (state == MachineState.KILL ||
    +                    state == MachineState.KILL_AND_RELAUNCH ||
    +                    state == MachineState.WAITING_FOR_WORKER_START) {
    +                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) {
    --- End diff --
    
    It looks like this used to be an `else if` but can now be outdented and move to its own line.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77326984
  
    --- Diff: storm-core/src/jvm/org/apache/storm/localizer/AsyncLocalizer.java ---
    @@ -0,0 +1,420 @@
    +/**
    + * 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.localizer;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.daemon.Shutdownable;
    +import org.apache.storm.daemon.supervisor.AdvancedFSOps;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is a wrapper around the Localizer class that provides the desired
    + * async interface to Slot.
    + * TODO once we have replaced the original supervisor merge this with
    + * Localizer and optimize them
    + */
    +public class AsyncLocalizer implements ILocalizer, Shutdownable {
    +    /**
    +     * A future that has already completed.
    +     */
    +    private static class AllDoneFuture implements Future<Void> {
    +
    +        @Override
    +        public boolean cancel(boolean mayInterruptIfRunning) {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isCancelled() {
    +            return false;
    +        }
    +
    +        @Override
    +        public boolean isDone() {
    +            return true;
    +        }
    +
    +        @Override
    +        public Void get() {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void get(long timeout, TimeUnit unit) {
    +            return null;
    +        }
    +
    +    }
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizer.class);
    +
    +    private final Localizer _localizer;
    +    private final ExecutorService _execService;
    +    private final boolean _isLocalMode;
    +    private final Map<String, Object> _conf;
    +    private final Map<String, LocalDownloadedResource> _basicPending;
    +    private final Map<String, LocalDownloadedResource> _blobPending;
    +    private final AdvancedFSOps _fsOps;
    +
    +    private class DownloadBaseBlobsDistributed implements Callable<Void> {
    +        private final String _topologyId;
    +        
    +        public DownloadBaseBlobsDistributed(String topologyId) {
    +            this._topologyId = topologyId;
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            File sr = new File(stormroot);
    +            if (sr.exists()) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    Utils.forceDelete(stormroot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            try {
    +                String stormJarKey = ConfigUtils.masterStormJarKey(_topologyId);
    +                String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +                String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +                String jarPath = ConfigUtils.supervisorStormJarPath(tmproot);
    +                String codePath = ConfigUtils.supervisorStormCodePath(tmproot);
    +                String confPath = ConfigUtils.supervisorStormConfPath(tmproot);
    +                FileUtils.forceMkdir(new File(tmproot));
    +                _fsOps.restrictDirectoryPermissions(tmproot);
    +                ClientBlobStore blobStore = Utils.getClientBlobStoreForSupervisor(_conf);
    +                try {
    +                    Utils.downloadResourcesAsSupervisor(stormJarKey, jarPath, blobStore);
    +                    Utils.downloadResourcesAsSupervisor(stormCodeKey, codePath, blobStore);
    +                    Utils.downloadResourcesAsSupervisor(stormConfKey, confPath, blobStore);
    +                } finally {
    +                    blobStore.shutdown();
    +                }
    +                Utils.extractDirFromJar(jarPath, ConfigUtils.RESOURCES_SUBDIR, tmproot);
    +                _fsOps.moveDirectoryPreferAtomic(new File(tmproot), new File(stormroot));
    +                SupervisorUtils.setupStormCodeDir(_conf, ConfigUtils.readSupervisorStormConf(_conf, _topologyId), stormroot);
    +                deleteAll = false;
    +            } finally {
    +                if (deleteAll) {
    +                    LOG.info("Failed to download basic resources for topology-id {}", _topologyId);
    +                    Utils.forceDelete(tmproot);
    +                    Utils.forceDelete(stormroot);
    +                }
    +            }
    +            return null;
    +        }
    +    }
    +    
    +    private class DownloadBaseBlobsLocal implements Callable<Void> {
    +        private final String _topologyId;
    +        
    +        public DownloadBaseBlobsLocal(String topologyId) {
    +            this._topologyId = topologyId;
    +        }
    +        
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            File sr = new File(stormroot);
    +            if (sr.exists()) {
    +                if (!_fsOps.supportsAtomicDirectoryMove()) {
    +                    LOG.warn("{} may have partially downloaded blobs, recovering", _topologyId);
    +                    Utils.forceDelete(stormroot);
    +                } else {
    +                    LOG.warn("{} already downloaded blobs, skipping", _topologyId);
    +                    return null;
    +                }
    +            }
    +            boolean deleteAll = true;
    +            String tmproot = ConfigUtils.supervisorTmpDir(_conf) + Utils.FILE_PATH_SEPARATOR + Utils.uuid();
    +            try {
    +                BlobStore blobStore = Utils.getNimbusBlobStore(_conf, null, null);
    +                FileOutputStream codeOutStream = null;
    +                FileOutputStream confOutStream = null;
    +                try {
    +                    FileUtils.forceMkdir(new File(tmproot));
    +                    String stormCodeKey = ConfigUtils.masterStormCodeKey(_topologyId);
    +                    String stormConfKey = ConfigUtils.masterStormConfKey(_topologyId);
    +                    String codePath = ConfigUtils.supervisorStormCodePath(tmproot);
    +                    String confPath = ConfigUtils.supervisorStormConfPath(tmproot);
    +                    codeOutStream = new FileOutputStream(codePath);
    +                    blobStore.readBlobTo(stormCodeKey, codeOutStream, null);
    +                    confOutStream = new FileOutputStream(confPath);
    +                    blobStore.readBlobTo(stormConfKey, confOutStream, null);
    +                } finally {
    +                    if (codeOutStream != null)
    +                        codeOutStream.close();
    +                    if (confOutStream != null)
    +                        codeOutStream.close();
    +                    blobStore.shutdown();
    +                }
    +
    +                ClassLoader classloader = Thread.currentThread().getContextClassLoader();
    +                String resourcesJar = AsyncLocalizer.resourcesJar();
    +                URL url = classloader.getResource(ConfigUtils.RESOURCES_SUBDIR);
    +
    +                String targetDir = tmproot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +
    +                if (resourcesJar != null) {
    +                    LOG.info("Extracting resources from jar at {} to {}", resourcesJar, targetDir);
    +                    Utils.extractDirFromJar(resourcesJar, ConfigUtils.RESOURCES_SUBDIR, stormroot);
    +                } else if (url != null) {
    +                    LOG.info("Copying resources at {} to {} ", url.toString(), targetDir);
    +                    if (url.getProtocol() == "jar") {
    +                        JarURLConnection urlConnection = (JarURLConnection) url.openConnection();
    +                        Utils.extractDirFromJar(urlConnection.getJarFileURL().getFile(), ConfigUtils.RESOURCES_SUBDIR, stormroot);
    +                    } else {
    +                        FileUtils.copyDirectory(new File(url.getFile()), (new File(targetDir)));
    +                    }
    +                }
    +                _fsOps.moveDirectoryPreferAtomic(new File(tmproot), new File(stormroot));
    +                SupervisorUtils.setupStormCodeDir(_conf, ConfigUtils.readSupervisorStormConf(_conf, _topologyId), stormroot);
    +                deleteAll = false;
    +            } finally {
    +                if (deleteAll) {
    +                    LOG.info("Failed to download basic resources for topology-id {}", _topologyId);
    +                    Utils.forceDelete(tmproot);
    +                    Utils.forceDelete(stormroot);
    +                }
    +            }
    +            return null;
    +        }
    +    }
    +    
    +    private class DownloadBlobs implements Callable<Void> {
    +        private final String _topologyId;
    +
    +        public DownloadBlobs(String topologyId) {
    +            _topologyId = topologyId;
    +        }
    +
    +        @Override
    +        public Void call() throws Exception {
    +            String stormroot = ConfigUtils.supervisorStormDistRoot(_conf, _topologyId);
    +            Map<String, Object> stormConf = ConfigUtils.readSupervisorStormConf(_conf, _topologyId);
    +                
    +            @SuppressWarnings("unchecked")
    +            Map<String, Map<String, Object>> blobstoreMap = (Map<String, Map<String, Object>>) stormConf.get(Config.TOPOLOGY_BLOBSTORE_MAP);
    +            String user = (String) stormConf.get(Config.TOPOLOGY_SUBMITTER_USER);
    +            String topoName = (String) stormConf.get(Config.TOPOLOGY_NAME);
    +
    +            List<LocalResource> localResourceList = new ArrayList<>();
    +            if (blobstoreMap != null) {
    +                List<LocalResource> tmp = SupervisorUtils.blobstoreMapToLocalresources(blobstoreMap);
    +                if (tmp != null) {
    +                    localResourceList.addAll(tmp);
    +                }
    +            }
    +
    +            StormTopology stormCode = ConfigUtils.readSupervisorTopology(_conf, _topologyId, _fsOps);
    +            List<String> dependencies = new ArrayList<>();
    +            if (stormCode.is_set_dependency_jars()) {
    +                dependencies.addAll(stormCode.get_dependency_jars());
    +            }
    +            if (stormCode.is_set_dependency_artifacts()) {
    +                dependencies.addAll(stormCode.get_dependency_artifacts());
    +            }
    +            for (String dependency : dependencies) {
    +                localResourceList.add(new LocalResource(dependency, false));
    +            }
    +            
    +            if (!localResourceList.isEmpty()) {
    +                File userDir = _localizer.getLocalUserFileCacheDir(user);
    +                if (!userDir.exists()) {
    +                    FileUtils.forceMkdir(userDir);
    +                }
    +                List<LocalizedResource> localizedResources = _localizer.getBlobs(localResourceList, user, topoName, userDir);
    +                _fsOps.setupBlobPermissions(userDir, user);
    +                for (LocalizedResource localizedResource : localizedResources) {
    +                    File rsrcFilePath = new File(localizedResource.getFilePath());
    +                    String keyName = rsrcFilePath.getName();
    +                    String blobSymlinkTargetName = new File(localizedResource.getCurrentSymlinkPath()).getName();
    +
    +                    String symlinkName = null;
    +                    Map<String, Object> blobInfo = blobstoreMap.get(keyName);
    --- End diff --
    
    If we use --jars or --dependencies and don't set blobstore map, blobstoreMap could be null.
    
    Below code resolves NPE issue and make it works, but it might be better to clarify that it is better to include dependencies to blobstore map or keep this as it is.
    
    ```
    if (blobstoreMap != null) {
                            Map<String, Object> blobInfo = blobstoreMap.get(keyName);
                            if (blobInfo != null && blobInfo.containsKey("localname")) {
                                symlinkName = (String) blobInfo.get("localname");
                            } else {
                                symlinkName = keyName;
                            }
                        } else {
                            // all things are from dependencies
                            symlinkName = keyName;
                        }
    ```


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76270910
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,494 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +    
    +    private class ProcessExitCallback implements Utils.ExitCodeCallable<Void> {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public Void call() throws Exception {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +            return null;
    +        }
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, LocalState localState, 
    +            ResourceIsolationInterface resourceIsolationManager, boolean recover) throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized(localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry: workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port +" "+ assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +        
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +    
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized(_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +        synchronized(_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            _localState.setApprovedWorkers(workersToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +    
    +    /**
    +     * Run the given command for profiling
    +     * @param command the command to run
    +     * @param env the environment to run the command
    +     * @param logPrefix the prefix to include in the logs
    +     * @param targetDir the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException on any error
    +     * @throws InterruptedException if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix, File targetDir) throws IOException, InterruptedException {
    +        Process p = Utils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +    
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String topologyId = _assignment.get_topology_id();
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, topologyId, _port);
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) topologyConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, topologyId, _port);
    +
    +        String workerPid = null;
    +        try (FileReader reader = new FileReader(str); BufferedReader br = new BufferedReader(reader)) {
    +            workerPid = br.readLine().trim();
    +        }
    +        
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        return runProfilingCommand(command, env, logPrefix, targetFile);
    +    }
    +    
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        if (action == ProfileAction.JMAP_DUMP) {
    +            return jmapDumpCmd(workerPid, targetDir);
    +        } else if (action == ProfileAction.JSTACK_DUMP) {
    +            return jstackDumpCmd(workerPid, targetDir);
    +        } else if (action == ProfileAction.JPROFILE_DUMP) {
    +            return jprofileDump(workerPid, targetDir);
    +        } else if (action == ProfileAction.JVM_RESTART) {
    +            return jprofileJvmRestart(workerPid);
    +        } else if (!stop && action == ProfileAction.JPROFILE_STOP) {
    +            return jprofileStart(workerPid);
    +        } else if (stop && action == ProfileAction.JPROFILE_STOP) {
    +            return jprofileStop(workerPid, targetDir);
    +        }
    +        return Lists.newArrayList();
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +    
    +    protected String jlp(String stormRoot, Map<String, Object> conf) {
    +        String resourceRoot = stormRoot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +        String os = System.getProperty("os.name").replaceAll("\\s+", "_");
    +        String arch = System.getProperty("os.arch");
    +        String archResourceRoot = resourceRoot + Utils.FILE_PATH_SEPARATOR + os + "-" + arch;
    +        String ret = archResourceRoot + Utils.CLASS_PATH_SEPARATOR + resourceRoot + Utils.CLASS_PATH_SEPARATOR + conf.get(Config.JAVA_LIBRARY_PATH);
    +        return ret;
    +    }
    +    
    +    @SuppressWarnings("unchecked")
    +    protected String getWorkerClassPath(String stormJar, Map<String, Object> topologyConf) {
    +        List<String> topoClasspath = new ArrayList<>();
    +        Object object = topologyConf.get(Config.TOPOLOGY_CLASSPATH);
    +
    +        // Will be populated only if STORM_TOPOLOGY_CLASSPATH_BEGINNING_ENABLED is set on Nimbus.
    +        // Allowed for extreme debugging.
    +        Object topologyClasspathFirst = topologyConf.get(Config.TOPOLOGY_CLASSPATH_BEGINNING);
    +        List<String> firstClasspathList = new ArrayList<>();
    +        if(topologyClasspathFirst instanceof List) {
    +            firstClasspathList.addAll((List<String>)topologyClasspathFirst);
    +        } else if (topologyClasspathFirst instanceof String) {
    +            firstClasspathList.add((String) topologyClasspathFirst);
    +        }
    +        LOG.debug("Topology Classpath Prefix: {}", firstClasspathList);
    +
    +        if (object instanceof List) {
    +            topoClasspath.addAll((List<String>) object);
    +        } else if (object instanceof String) {
    +            topoClasspath.add((String) object);
    +        }
    +        LOG.debug("Topology specific classpath is {}", object);
    +
    +        String classPath = Utils.addToClasspath(firstClasspathList, Arrays.asList(Utils.workerClasspath()));
    +        String classAddPath = Utils.addToClasspath(classPath, Arrays.asList(stormJar));
    +        return Utils.addToClasspath(classAddPath, topoClasspath);
    +    }
    +    
    +    private static String substituteChildOptsInternal(String string,  String workerId, String stormId, int port, int memOnheap) {
    +        if (StringUtils.isNotBlank(string)){
    +            string = string.replace("%ID%", String.valueOf(port));
    +            string = string.replace("%WORKER-ID%", workerId);
    +            string = string.replace("%TOPOLOGY-ID%", stormId);
    +            string = string.replace("%WORKER-PORT%", String.valueOf(port));
    +            string = string.replace("%HEAP-MEM%", String.valueOf(memOnheap));
    +        }
    +        return string;
    +    }
    +    
    +    protected List<String> substituteChildopts(Object value, String workerId, String stormId, int port, int memOnheap) {
    +        List<String> rets = new ArrayList<>();
    +        if (value instanceof String) {
    +            String string = substituteChildOptsInternal((String) value,  workerId, stormId, port, memOnheap);
    +            if (StringUtils.isNotBlank(string)){
    +                String[] strings = string.split("\\s+");
    +                rets.addAll(Arrays.asList(strings));
    +            }
    +        } else if (value instanceof List) {
    +            @SuppressWarnings("unchecked")
    +            List<Object> objects = (List<Object>) value;
    +            for (Object object : objects) {
    +                String str = substituteChildOptsInternal((String) object,  workerId, stormId, port, memOnheap);
    +                if (StringUtils.isNotBlank(str)){
    +                    rets.add(str);
    +                }
    +            }
    +        }
    +        return rets;
    +    }
    +
    +    /**
    +     * Launch the worker process (non-blocking)
    +     * @param command the command to run
    +     * @param env the environment to run the command
    +     * @param processExitcallback a callback for when the process exits
    +     * @param logPrefix the prefix to include in the logs
    +     * @param targetDir the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException on any error
    +     */
    +    protected void launchWorkerProcess(List<String> command, Map<String, String> env, 
    +            String logPrefix, Utils.ExitCodeCallable<Void> processExitCallback, File targetDir) throws IOException {
    +        Utils.launchProcess(command, env, logPrefix, processExitCallback, targetDir);
    +    }
    +    
    +    @Override
    +    public void launch() throws IOException {
    --- End diff --
    
    It will be good add docs around the launch method . In current 1.x branch this code takes sometime to understand


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77296684
  
    --- Diff: storm-core/test/jvm/org/apache/storm/daemon/supervisor/BasicContainerTest.java ---
    @@ -0,0 +1,459 @@
    +package org.apache.storm.daemon.supervisor;
    +
    +import static org.junit.Assert.*;
    +import static org.mockito.Mockito.*;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.junit.Test;
    +
    +public class BasicContainerTest {
    +    public static class CommandRun {
    +        final List<String> cmd;
    +        final Map<String, String> env;
    +        final File pwd;
    +        
    +        public CommandRun(List<String> cmd, Map<String, String> env, File pwd) {
    +            this.cmd = cmd;
    +            this.env = env;
    +            this.pwd = pwd;
    +        }
    +    }
    +    
    +    public static class MockBasicContainer extends BasicContainer {
    +        public final List<CommandRun> profileCmds = new ArrayList<>();
    +        public final List<CommandRun> workerCmds = new ArrayList<>();
    +        
    +        public MockBasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf,
    +                String supervisorId, LocalState localState, ResourceIsolationInterface resourceIsolationManager,
    +                boolean recover) throws IOException {
    +            super(port, assignment, conf, supervisorId, localState, resourceIsolationManager, recover);
    +        }
    +        
    +        public MockBasicContainer(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +                Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +                ResourceIsolationInterface resourceIsolationManager, LocalState localState,
    +                String profileCmd) throws IOException {
    +            super(ops, port, assignment, conf, topoConf, supervisorId, resourceIsolationManager, localState, profileCmd);
    +        }
    +        
    +        @Override
    +        protected Map<String, Object> readTopoConf() throws IOException {
    +            return new HashMap<>();
    +        }
    +        
    +        @Override
    +        public void createNewWorkerId() {
    +            super.createNewWorkerId();
    +        }
    +        
    +        @Override
    +        public List<String> substituteChildopts(Object value, int memOnheap) {
    +            return super.substituteChildopts(value, memOnheap);
    +        }
    +               
    +        @Override
    +        protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +                File targetDir) throws IOException, InterruptedException {
    +            profileCmds.add(new CommandRun(command, env, targetDir));
    +            return true;
    +        }
    +        
    +        @Override
    +        protected void launchWorkerProcess(List<String> command, Map<String, String> env, String logPrefix,
    +                ExitCodeCallback processExitCallback, File targetDir) throws IOException {
    +            workerCmds.add(new CommandRun(command, env, targetDir));
    +        }
    +        
    +        @Override
    +        protected String javaCmd(String cmd) {
    +            //avoid system dependent things
    +            return cmd;
    +        }
    +        
    +        @Override
    +        protected List<String> frameworkClasspath() {
    +            //We are not really running anything so make this
    +            // simple to check for
    +            return Arrays.asList("FRAMEWORK_CP");
    +        }
    +        
    +        @Override
    +        protected String javaLibraryPath(String stormRoot, Map<String, Object> conf) {
    +            return "JLP";
    +        }
    +    }
    +    
    +    @Test
    +    public void testCreateNewWorkerId() throws Exception {
    +        final String topoId = "test_topology";
    +        final int port = 8080;
    +        LocalAssignment la = new LocalAssignment();
    +        la.set_topology_id(topoId);
    +        
    +        AdvancedFSOps ops = mock(AdvancedFSOps.class);
    +        
    +        LocalState ls = mock(LocalState.class);
    +        
    +        MockBasicContainer mc = new MockBasicContainer(ops, port, la, new HashMap<String, Object>(), 
    +                new HashMap<String, Object>(), "SUPERVISOR", null, ls, "profile");
    +        
    +        mc.createNewWorkerId();
    +        
    +        assertNotNull(mc._workerId);
    +        verify(ls).getApprovedWorkers();
    +        Map<String, Integer> expectedNewState = new HashMap<String, Integer>();
    +        expectedNewState.put(mc._workerId, port);
    +        verify(ls).setApprovedWorkers(expectedNewState);
    +    }
    +    
    +    @Test
    +    public void testRecovery() throws Exception {
    +        final String topoId = "test_topology";
    +        final String workerId = "myWorker";
    +        final int port = 8080;
    +        LocalAssignment la = new LocalAssignment();
    +        la.set_topology_id(topoId);
    +        
    +        Map<String, Integer> workerState = new HashMap<String, Integer>();
    +        workerState.put(workerId, port);
    +        
    +        LocalState ls = mock(LocalState.class);
    +        when(ls.getApprovedWorkers()).thenReturn(workerState);
    +        
    +        MockBasicContainer mc = new MockBasicContainer(port, la, new HashMap<String, Object>(), 
    +                "SUPERVISOR", ls, null, true);
    +        
    +        assertEquals(workerId, mc._workerId);
    +    }
    +    
    +    @Test
    +    public void testRecoveryMiss() throws Exception {
    +        final String topoId = "test_topology";
    +        final int port = 8080;
    +        LocalAssignment la = new LocalAssignment();
    +        la.set_topology_id(topoId);
    +        
    +        Map<String, Integer> workerState = new HashMap<String, Integer>();
    +        workerState.put("somethingelse", port+1);
    +        
    +        LocalState ls = mock(LocalState.class);
    +        when(ls.getApprovedWorkers()).thenReturn(workerState);
    +        
    +        try {
    +            new MockBasicContainer(port, la, new HashMap<String, Object>(), 
    +                    "SUPERVISOR", ls, null, true);
    +            fail("Container recovered worker incorrectly");
    +        } catch (ContainerRecoveryException e) {
    +            //Expected
    +        }
    +    }
    +    
    +    @Test
    +    public void testCleanUp() throws Exception {
    +        final String topoId = "test_topology";
    +        final int port = 8080;
    +        final String workerId = "worker-id";
    +        LocalAssignment la = new LocalAssignment();
    +        la.set_topology_id(topoId);
    +        
    +        AdvancedFSOps ops = mock(AdvancedFSOps.class);
    +        
    +        Map<String, Integer> workerState = new HashMap<String, Integer>();
    +        workerState.put(workerId, port);
    +        
    +        LocalState ls = mock(LocalState.class);
    +        when(ls.getApprovedWorkers()).thenReturn(new HashMap<>(workerState));
    +        
    +        MockBasicContainer mc = new MockBasicContainer(ops, port, la, new HashMap<String, Object>(), 
    +                new HashMap<String, Object>(), "SUPERVISOR", null, ls, "profile");
    +        mc._workerId = workerId;
    +        
    +        mc.cleanUp();
    +        
    +        assertNull(mc._workerId);
    +        verify(ls).getApprovedWorkers();
    +        Map<String, Integer> expectedNewState = new HashMap<String, Integer>();
    +        verify(ls).setApprovedWorkers(expectedNewState);
    +    }
    +    
    +    @Test
    +    public void testRunProfiling() throws Exception {
    +        final long pid = 100;
    +        final String topoId = "test_topology";
    +        final int port = 8080;
    +        final String workerId = "worker-id";
    +        final String stormLocal = ContainerTest.asAbsPath("tmp", "testing");
    +        final String topoRoot = ContainerTest.asAbsPath(stormLocal, topoId, String.valueOf(port));
    +        final File workerArtifactsPid = ContainerTest.asAbsFile(topoRoot, "worker.pid");
    +        
    +        final Map<String, Object> superConf = new HashMap<>();
    +        superConf.put(Config.STORM_LOCAL_DIR, stormLocal);
    +        superConf.put(Config.STORM_WORKERS_ARTIFACTS_DIR, stormLocal);
    +        
    +        LocalAssignment la = new LocalAssignment();
    +        la.set_topology_id(topoId);
    +        
    +        AdvancedFSOps ops = mock(AdvancedFSOps.class);
    +        when(ops.slurpString(workerArtifactsPid)).thenReturn(String.valueOf(pid));
    +        
    +        LocalState ls = mock(LocalState.class);
    +        
    +        MockBasicContainer mc = new MockBasicContainer(ops, port, la, superConf, 
    +                new HashMap<String, Object>(), "SUPERVISOR", null, ls, "profile");
    +        mc._workerId = workerId;
    +        
    +        //HEAP DUMP
    +        ProfileRequest req = new ProfileRequest();
    +        req.set_action(ProfileAction.JMAP_DUMP);
    +        
    +        mc.runProfiling(req, false);
    +        
    +        assertEquals(1, mc.profileCmds.size());
    +        CommandRun cmd = mc.profileCmds.get(0);
    +        mc.profileCmds.clear();
    +        assertEquals(Arrays.asList("profile", String.valueOf(pid), "jmap", topoRoot), cmd.cmd);
    +        assertEquals(new File(topoRoot), cmd.pwd);
    +        
    +        //JSTACK DUMP
    +        req.set_action(ProfileAction.JSTACK_DUMP);
    +        
    +        mc.runProfiling(req, false);
    +        
    +        assertEquals(1, mc.profileCmds.size());
    +        cmd = mc.profileCmds.get(0);
    +        mc.profileCmds.clear();
    +        assertEquals(Arrays.asList("profile", String.valueOf(pid), "jstack", topoRoot), cmd.cmd);
    +        assertEquals(new File(topoRoot), cmd.pwd);
    +        
    +        //RESTART
    +        req.set_action(ProfileAction.JVM_RESTART);
    +        
    +        mc.runProfiling(req, false);
    +        
    +        assertEquals(1, mc.profileCmds.size());
    +        cmd = mc.profileCmds.get(0);
    +        mc.profileCmds.clear();
    +        assertEquals(Arrays.asList("profile", String.valueOf(pid), "kill"), cmd.cmd);
    +        assertEquals(new File(topoRoot), cmd.pwd);
    +        
    +        //JPROFILE DUMP
    +        req.set_action(ProfileAction.JPROFILE_DUMP);
    +        
    +        mc.runProfiling(req, false);
    +        
    +        assertEquals(1, mc.profileCmds.size());
    +        cmd = mc.profileCmds.get(0);
    +        mc.profileCmds.clear();
    +        assertEquals(Arrays.asList("profile", String.valueOf(pid), "dump", topoRoot), cmd.cmd);
    +        assertEquals(new File(topoRoot), cmd.pwd);
    +        
    +        //JPROFILE START
    +        req.set_action(ProfileAction.JPROFILE_STOP);
    --- End diff --
    
    JPROFILE_STOP -> JPROFILE_START?


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    Not sure why travis has so many issues with the build and maven but here is my travis setup on the same branch with the build passing.
    
    https://travis-ci.org/revans2/incubator-storm/builds/160266388


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77143604
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,314 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.Writer;
    +import java.nio.file.FileSystems;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    +    private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
    +    
    +    /**
    +     * Factory to create a new AdvancedFSOps
    +     * @param conf the configuration of the process
    +     * @return the appropriate instance of the class for this config and environment.
    +     */
    +    public static AdvancedFSOps make(Map<String, Object> conf) {
    +        if (Utils.isOnWindows()) {
    +            return new AdvancedWindowsFSOps(conf);
    +        }
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new AdvancedRunAsUserFSOps(conf);
    +        }
    +        return new AdvancedFSOps();
    +    }
    +    
    +    private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
    +        private final Map<String, Object> _conf;
    +        
    +        public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
    +            if (Utils.isOnWindows()) {
    +                throw new UnsupportedOperationException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +            _conf = conf;
    +        }
    +        
    +        @Override
    +        public void setupBlobPermissions(File path, String user) throws IOException {
    +            String logPrefix = "setup blob permissions for " + path;
    +            SupervisorUtils.processLauncherAndWait(_conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
    +        }
    +        
    +        @Override
    +        public void deleteIfExists(File path, String user, String logPrefix) throws IOException {
    +            String absolutePath = path.getAbsolutePath();
    +            LOG.info("Deleting path {}", absolutePath);
    +            if (user == null) {
    +                user = Files.getOwner(path.toPath()).getName();
    +            }
    +            List<String> commands = new ArrayList<>();
    +            commands.add("rmr");
    +            commands.add(absolutePath);
    +            SupervisorUtils.processLauncherAndWait(_conf, user, commands, null, logPrefix);
    +            if (Utils.checkFileExists(absolutePath)) {
    +                throw new RuntimeException(path + " was not deleted.");
    +            }
    +        }
    +        
    +        @Override
    +        public void setupStormCodeDir(Map<String, Object> topologyConf, File path) throws IOException {
    +            SupervisorUtils.setupStormCodeDir(_conf, topologyConf, path.getCanonicalPath());
    +        }
    +    }
    +    
    +    /**
    +     * Operations that need to override the default ones when running on Windows
    +     *
    +     */
    +    private static class AdvancedWindowsFSOps extends AdvancedFSOps {
    +
    +        public AdvancedWindowsFSOps(Map<String, Object> conf) {
    +            if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +                throw new RuntimeException("ERROR: Windows doesn't support running workers as different users yet");
    +            }
    +        }
    +        
    +        @Override
    +        public void restrictDirectoryPermissions(String dir) throws IOException {
    +            //NOOP, if windows gets support for run as user we will need to find a way to suppor this
    +        }
    +        
    +        @Override
    +        public void moveDriectoryPreferAtomic(File fromDir, File toDir) throws IOException {
    --- End diff --
    
    This function name is misspelled


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77417198
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,318 @@
    +/**
    + * 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.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.localizer.ILocalizer;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.scheduler.ISupervisor;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this(supervisor.getConf(), supervisor.getStormClusterState(), supervisor.getEventManger(),
    +                supervisor.getAssignmentId(), supervisor.getiSupervisor(),
    +                supervisor.getAsyncLocalizer(), supervisor.getHostName(),
    +                supervisor.getLocalState(), supervisor.getStormClusterState(),
    +                supervisor.getCurrAssignment(), supervisor.getSharedContext());
    +    }
    +    
    +    public ReadClusterState(Map<String, Object> superConf, IStormClusterState stormClusterState,
    +            EventManager syncSupEventManager, String assignmentId, ISupervisor iSuper,
    +            ILocalizer localizer, String host, LocalState localState,
    +            IStormClusterState clusterState, AtomicReference<Map<Long, LocalAssignment>> cachedAssignments,
    +            IContext sharedContext) throws Exception{
    +        this.superConf = superConf;
    +        this.stormClusterState = stormClusterState;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.assignmentVersions = new AtomicReference<Map<String, VersionedData<Assignment>>>(new HashMap<String, VersionedData<Assignment>>());
    +        this.assignmentId = assignmentId;
    +        this.iSuper = iSuper;
    +        this.localizer = localizer;
    +        this.host = host;
    +        this.localState = localState;
    +        this.clusterState = clusterState;
    +        this.cachedAssignments = cachedAssignments;
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, sharedContext);
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        Slot slot = new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +        slot.start();
    +        return slot;
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    +                            if (actions == null) {
    +                                actions = new HashSet<>();
    +                                filtered.put(port.intValue(), actions);
    +                            }
    +                            actions.add(new TopoProfileAction(topoId, req));
    +                        }
    +                    }
    +                }
    +            }
    +            
    +            for (Integer port: allPorts) {
    +                Slot slot = slots.get(port);
    +                if (slot == null) {
    +                    slot = mkSlot(port);
    +                    slots.put(port, slot);
    +                }
    +                slot.setNewAssignment(allAssignments.get(port));
    +                slot.addProfilerActions(filtered.get(port));
    +            }
    +            
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +    }
    +    
    +    protected Map<String, VersionedData<Assignment>> getAssignmentsSnapshot(IStormClusterState stormClusterState, List<String> topoIds,
    +            Map<String, VersionedData<Assignment>> localAssignmentVersion, Runnable callback) throws Exception {
    +        Map<String, VersionedData<Assignment>> updateAssignmentVersion = new HashMap<>();
    +        for (String topoId : topoIds) {
    +            Integer recordedVersion = -1;
    +            Integer version = stormClusterState.assignmentVersion(topoId, callback);
    +            VersionedData<Assignment> locAssignment = localAssignmentVersion.get(topoId);
    +            if (locAssignment != null) {
    +                recordedVersion = locAssignment.getVersion();
    +            }
    +            if (version == null) {
    +                // ignore
    +            } else if (version == recordedVersion) {
    +                updateAssignmentVersion.put(topoId, locAssignment);
    +            } else {
    +                VersionedData<Assignment> assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback);
    +                updateAssignmentVersion.put(topoId, assignmentVersion);
    +            }
    +        }
    +        return updateAssignmentVersion;
    +    }
    +    
    +    protected Map<String, List<ProfileRequest>> getProfileActions(IStormClusterState stormClusterState, List<String> stormIds) throws Exception {
    +        Map<String, List<ProfileRequest>> ret = new HashMap<String, List<ProfileRequest>>();
    +        for (String stormId : stormIds) {
    +            List<ProfileRequest> profileRequests = stormClusterState.getTopologyProfileRequests(stormId);
    +            ret.put(stormId, profileRequests);
    +        }
    +        return ret;
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readAssignments(Map<String, VersionedData<Assignment>> assignmentsSnapshot,
    +            String assignmentId, AtomicInteger retries) {
    +        try {
    +            Map<Integer, LocalAssignment> portLA = new HashMap<Integer, LocalAssignment>();
    +            for (Map.Entry<String, VersionedData<Assignment>> assignEntry : assignmentsSnapshot.entrySet()) {
    +                String topoId = assignEntry.getKey();
    +                Assignment assignment = assignEntry.getValue().getData();
    +
    +                Map<Integer, LocalAssignment> portTasks = readMyExecutors(topoId, assignmentId, assignment);
    +
    +                for (Map.Entry<Integer, LocalAssignment> entry : portTasks.entrySet()) {
    +
    +                    Integer port = entry.getKey();
    +
    +                    LocalAssignment la = entry.getValue();
    +
    +                    if (!portLA.containsKey(port)) {
    +                        portLA.put(port, la);
    +                    } else {
    +                        throw new RuntimeException("Should not have multiple topologies assigned to one port");
    +                    }
    +                }
    +            }
    +            retries.set(0);
    +            return portLA;
    +        } catch (RuntimeException e) {
    +            if (retries.get() > 2) {
    +                throw e;
    +            } else {
    +                retries.addAndGet(1);
    +            }
    +            LOG.warn("{} : retrying {} of 3", e.getMessage(), retries.get());
    +            return null;
    +        }
    +    }
    +    
    +    protected Map<Integer, LocalAssignment> readMyExecutors(String stormId, String assignmentId, Assignment assignment) {
    +        Map<Integer, LocalAssignment> portTasks = new HashMap<>();
    +        Map<Long, WorkerResources> slotsResources = new HashMap<>();
    +        Map<NodeInfo, WorkerResources> nodeInfoWorkerResourcesMap = assignment.get_worker_resources();
    +        if (nodeInfoWorkerResourcesMap != null) {
    +            for (Map.Entry<NodeInfo, WorkerResources> entry : nodeInfoWorkerResourcesMap.entrySet()) {
    +                if (entry.getKey().get_node().equals(assignmentId)) {
    +                    Set<Long> ports = entry.getKey().get_port();
    +                    for (Long port : ports) {
    +                        slotsResources.put(port, entry.getValue());
    +                    }
    +                }
    +            }
    +        }
    +        Map<List<Long>, NodeInfo> executorNodePort = assignment.get_executor_node_port();
    +        if (executorNodePort != null) {
    +            for (Map.Entry<List<Long>, NodeInfo> entry : executorNodePort.entrySet()) {
    +                if (entry.getValue().get_node().equals(assignmentId)) {
    +                    for (Long port : entry.getValue().get_port()) {
    +                        LocalAssignment localAssignment = portTasks.get(port.intValue());
    +                        if (localAssignment == null) {
    +                            List<ExecutorInfo> executors = new ArrayList<ExecutorInfo>();
    +                            localAssignment = new LocalAssignment(stormId, executors);
    +                            if (slotsResources.containsKey(port)) {
    +                                localAssignment.set_resources(slotsResources.get(port));
    +                            }
    +                            portTasks.put(port.intValue(), localAssignment);
    +                        }
    +                        List<ExecutorInfo> executorInfoList = localAssignment.get_executors();
    +                        executorInfoList.add(new ExecutorInfo(entry.getKey().get(0).intValue(), entry.getKey().get(entry.getKey().size() - 1).intValue()));
    +                    }
    +                }
    +            }
    +        }
    +        return portTasks;
    +    }
    +
    +    public synchronized void shutdownAllWorkers() {
    +        for (Slot slot: slots.values()) {
    +            try {
    +                slot.setNewAssignment(null);
    +            } catch (Exception e) {
    +                LOG.error("Error trying to shutdown workers in {}", slot, e);
    +            }
    --- End diff --
    
    :) good catch


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76161402
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,417 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.AdvancedFSOps;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill the processes in this container nicely.
    +     * kill -15 equivalent
    +     * @throws IOException on any error
    +     */
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Kill the processes in this container violently.
    +     * kill -9 equivalent
    +     * @throws IOException on any error
    +     */
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq"+pid);
    +            pb.redirectError(Redirect.INHERIT);
    +            Process p = pb.start();
    +            try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +                //TODO would be really nice to parse out the user
    +                if (in.readLine() != null) {
    +                    return true;
    +                }
    +            }
    +            return false;
    +        }
    +        //Not on Windows assume POSIX
    +        ProcessBuilder pb = new ProcessBuilder("ps", "-o", "user", "-p", String.valueOf(pid));
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            String first = in.readLine();
    +            assert("USER".equals(first));
    +            String processUser;
    +            while ((processUser = in.readLine()) != null) {
    +                if (user.equals(processUser)) {
    +                    return true;
    +                } else {
    +                    LOG.info("Found {} running as {}, but expected it to be {}", pid, processUser, user);
    --- End diff --
    
    break; ?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77144436
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java ---
    @@ -0,0 +1,314 @@
    +/**
    + * 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.File;
    +import java.io.FileNotFoundException;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.Writer;
    +import java.nio.file.FileSystems;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.StandardCopyOption;
    +import java.nio.file.attribute.PosixFilePermission;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class AdvancedFSOps {
    --- End diff --
    
    If this class is supposed to be subclassed before use, why not make it abstract?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77150790
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +
    +    //For testing purposes
    +    public BasicContainer(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager, LocalState localState,
    +            String profileCmd) throws IOException {
    +        super(ops, port, assignment, conf, topoConf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +        _profileCmd = profileCmd;
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, String supervisorId,
    +            LocalState localState, ResourceIsolationInterface resourceIsolationManager, boolean recover)
    +            throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                LOG.info("Recovered Worker {}", wid);
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException(
    +                    "Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            removeWorkersOn(workerToPort, _port);
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +            LOG.info("Created Worker ID {}", _workerId);
    +        }
    +    }
    +
    +    private static void removeWorkersOn(Map<String, Integer> workerToPort, int _port) {
    +        for (Iterator<Entry<String, Integer>> i = workerToPort.entrySet().iterator(); i.hasNext();) {
    +            Entry<String, Integer> found = i.next();
    +            if (_port == found.getValue().intValue()) {
    +                i.remove();
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUpForRestart() throws IOException {
    +        super.cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            removeWorkersOn(workersToPort, _port);
    +            _localState.setApprovedWorkers(workersToPort);
    +            LOG.info("Removed Worker ID {}", _workerId);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        setup();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = _ops.slurpString(new File(str)).trim();
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        if (command.size() > 0) {
    +            return runProfilingCommand(command, env, logPrefix, targetFile);
    +        }
    +        LOG.warn("PROFILING REQUEST NOT SUPPORTED {} IGNORED...", request);
    +        return true;
    +    }
    +
    +    /**
    +     * Get the command to run when doing profiling
    +     * @param action the profiling action to perform
    +     * @param stop if this is meant to stop the profiling or start it
    +     * @param workerPid the PID of the process to profile
    +     * @param targetDir the current working directory of the worker process
    +     * @return the command to run for profiling.
    +     */
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        switch(action) {
    +            case JMAP_DUMP:
    +                return jmapDumpCmd(workerPid, targetDir);
    +            case JSTACK_DUMP:
    +                return jstackDumpCmd(workerPid, targetDir);
    +            case JPROFILE_DUMP:
    +                return jprofileDump(workerPid, targetDir);
    +            case JVM_RESTART:
    +                return jprofileJvmRestart(workerPid);
    +            case JPROFILE_STOP:
    +                if (stop) {
    +                    return jprofileStop(workerPid, targetDir);
    +                }
    +                return jprofileStart(workerPid);
    +            default:
    +                return Lists.newArrayList();
    +        }
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +
    +    /**
    +     * Compute the java.library.path that should be used for the worker.
    +     * This helps it to load JNI libraries that are packaged in the uber jar.
    +     * @param stormRoot the root directory of the worker process
    +     * @param conf the config for the supervisor.
    +     * @return the java.library.path/LD_LIBRARY_PATH to use so native libraries load correctly.
    +     */
    +    protected String jlp(String stormRoot, Map<String, Object> conf) {
    --- End diff --
    
    Nitpick: Consider not abbreviating this. It's obvious what it is here, but probably less so when called from elsewhere in the code.


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76476353
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,437 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = -1;
    +        _assignment = null;
    +        _workerId = workerId;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.warn("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("tasklist", "/nh", "/fi", "pid eq"+pid);
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            if (in.readLine() != null) {
    +                ret = true;
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    private boolean isPosixProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    +        ProcessBuilder pb = new ProcessBuilder("ps", "-o", "user", "-p", String.valueOf(pid));
    +        pb.redirectError(Redirect.INHERIT);
    +        Process p = pb.start();
    +        try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()))) {
    +            String first = in.readLine();
    +            assert("USER".equals(first));
    +            String processUser;
    +            while ((processUser = in.readLine()) != null) {
    +                if (user.equals(processUser)) {
    +                    ret = true;
    +                    break;
    +                } else {
    +                    LOG.info("Found {} running as {}, but expected it to be {}", pid, processUser, user);
    +                }
    +            }
    +        }
    +        return ret;
    +    }
    +    
    +    @Override
    +    public boolean areAllProcessesDead() throws IOException {
    +        Set<Long> pids = getAllPids();
    +        String user = getWorkerUser();
    +        
    +        boolean allDead = true;
    +        for (Long pid: pids) {
    +            if (!isProcessAlive(pid, user)) {
    +                LOG.warn("{}: PID {} is dead", _workerId, pid);
    +            } else {
    +                allDead = false;
    +                break;
    +            }
    +        }
    +        return allDead;
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +    }
    +
    +    /**
    +     * Setup the container to run.  By default this creates the needed directories/links in the
    +     * local file system
    +     * PREREQUISITE: All needed blobs and topology, jars/configs have been downloaded and
    +     * placed in the appropriate locations
    +     * @throws IOException on any error
    +     */
    +    protected void setup() throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    +        }
    +        final String topologyId = _assignment.get_topology_id();
    +        if (!SupervisorUtils.doRequiredTopoFilesExist(_conf, topologyId)) {
    +            LOG.info("Missing topology storm code, so can't launch  worker with assignment {} for this supervisor {} on port {} with id {}", _assignment,
    +                    _supervisorId, _port, _workerId);
    +            throw new IllegalStateException("Not all needed files are here!!!!");
    +        }
    +        String pidsPath = ConfigUtils.workerPidsRoot(_conf, _workerId);
    +        String hbPath = ConfigUtils.workerHeartbeatsRoot(_conf, _workerId);
    +    
    +        FileUtils.forceMkdir(new File(pidsPath));
    +        FileUtils.forceMkdir(new File(ConfigUtils.workerTmpRoot(_conf, _workerId)));
    +        FileUtils.forceMkdir(new File(hbPath));
    +    
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        String user = (String) topologyConf.get(Config.TOPOLOGY_SUBMITTER_USER);
    +        writeLogMetadata(topologyConf, user, topologyId);
    +        ConfigUtils.setWorkerUserWSE(_conf, _workerId, user);
    +        createArtifactsLink(topologyId);
    +    
    +        createBlobstoreLinks(topologyId);
    +    }
    +    
    +    /**
    +     * Write out the file used by the log viewer to allow/reject log access
    +     * @param topologyConf the config for the topology
    +     * @param user the user this is going to run as
    +     * @param topologyId the id of the topology
    +     * @throws IOException on any error
    +     */
    +    @SuppressWarnings("unchecked")
    +    protected void writeLogMetadata(Map<String, Object> topologyConf, String user, String topologyId) throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    +        }
    +        Map<String, Object> data = new HashMap<>();
    +        data.put(Config.TOPOLOGY_SUBMITTER_USER, user);
    +        data.put("worker-id", _workerId);
    +
    +        Set<String> logsGroups = new HashSet<>();
    +        //for supervisor-test
    +        if (topologyConf.get(Config.LOGS_GROUPS) != null) {
    +            List<String> groups = (List<String>) topologyConf.get(Config.LOGS_GROUPS);
    +            for (String group : groups){
    +                logsGroups.add(group);
    +            }
    +        }
    +        if (topologyConf.get(Config.TOPOLOGY_GROUPS) != null) {
    +            List<String> topGroups = (List<String>) topologyConf.get(Config.TOPOLOGY_GROUPS);
    +            logsGroups.addAll(topGroups);
    +        }
    +        data.put(Config.LOGS_GROUPS, logsGroups.toArray());
    +
    +        Set<String> logsUsers = new HashSet<>();
    +        if (topologyConf.get(Config.LOGS_USERS) != null) {
    +            List<String> logUsers = (List<String>) topologyConf.get(Config.LOGS_USERS);
    +            for (String logUser : logUsers){
    +                logsUsers.add(logUser);
    +            }
    +        }
    +        if (topologyConf.get(Config.TOPOLOGY_USERS) != null) {
    +            List<String> topUsers = (List<String>) topologyConf.get(Config.TOPOLOGY_USERS);
    +            for (String logUser : topUsers){
    +                logsUsers.add(logUser);
    +            }
    +        }
    +        data.put(Config.LOGS_USERS, logsUsers.toArray());
    +
    +        File file = ConfigUtils.getLogMetaDataFile(_conf, topologyId, _port);
    +        File parent = file.getParentFile();
    +        if (!Utils.checkFileExists(parent)) {
    +            FileUtils.forceMkdir(file.getParentFile());
    +            _ops.setupStormCodeDir(ConfigUtils.readSupervisorStormConf(_conf, topologyId), file.getParentFile().getCanonicalPath());
    +        }
    +        Yaml yaml = new Yaml();
    +        try (FileWriter writer = new FileWriter(file)) {
    +            yaml.dump(data, writer);
    +        }
    +    }
    +    
    +    /**
    +     * Create symlink from the containers directory/artifacts to the artifacts directory
    +     * @param topologyId the id of the topology this is for
    +     * @throws IOException on any error
    +     */
    +    protected void createArtifactsLink(String topologyId) throws IOException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot setup a container recovered with just a worker id");
    --- End diff --
    
    copy/paste?


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76270332
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,494 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +    
    +    private class ProcessExitCallback implements Utils.ExitCodeCallable<Void> {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public Void call() throws Exception {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +            return null;
    +        }
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, LocalState localState, 
    +            ResourceIsolationInterface resourceIsolationManager, boolean recover) throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized(localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry: workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port +" "+ assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +        
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +    
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized(_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +        synchronized(_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            _localState.setApprovedWorkers(workersToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +    
    +    /**
    +     * Run the given command for profiling
    +     * @param command the command to run
    +     * @param env the environment to run the command
    +     * @param logPrefix the prefix to include in the logs
    +     * @param targetDir the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException on any error
    +     * @throws InterruptedException if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix, File targetDir) throws IOException, InterruptedException {
    +        Process p = Utils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +    
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String topologyId = _assignment.get_topology_id();
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, topologyId, _port);
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) topologyConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, topologyId, _port);
    +
    +        String workerPid = null;
    +        try (FileReader reader = new FileReader(str); BufferedReader br = new BufferedReader(reader)) {
    +            workerPid = br.readLine().trim();
    +        }
    +        
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        return runProfilingCommand(command, env, logPrefix, targetFile);
    +    }
    +    
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        if (action == ProfileAction.JMAP_DUMP) {
    +            return jmapDumpCmd(workerPid, targetDir);
    +        } else if (action == ProfileAction.JSTACK_DUMP) {
    +            return jstackDumpCmd(workerPid, targetDir);
    +        } else if (action == ProfileAction.JPROFILE_DUMP) {
    +            return jprofileDump(workerPid, targetDir);
    +        } else if (action == ProfileAction.JVM_RESTART) {
    +            return jprofileJvmRestart(workerPid);
    +        } else if (!stop && action == ProfileAction.JPROFILE_STOP) {
    +            return jprofileStart(workerPid);
    +        } else if (stop && action == ProfileAction.JPROFILE_STOP) {
    +            return jprofileStop(workerPid, targetDir);
    +        }
    +        return Lists.newArrayList();
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    --- End diff --
    
    should these "jmap" should replaced by ProfileAction.JMAP_DUMP


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76160977
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,417 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.AdvancedFSOps;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected String _workerId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) {
    +        _port = port;
    +        _assignment = assignment;
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        _ops = AdvancedFSOps.mk(conf);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    --- End diff --
    
    Do we want to log a force kill like we do a "nice" kill?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77415016
  
    --- Diff: storm-core/src/jvm/org/apache/storm/utils/Utils.java ---
    @@ -1990,31 +1995,26 @@ protected void forceDeleteImpl(String path) throws IOException {
         }
     
         /**
    -     * Creates a symbolic link to the target
    +     * Creates a symbolic link to the target and force the creation if the target already exists
          * @param dir the parent directory of the link
          * @param targetDir the parent directory of the link's target
    -     * @param filename the file name of the link
          * @param targetFilename the file name of the links target
    +     * @param filename the file name of the link
          * @throws IOException
          */
    -    public static void createSymlink(String dir, String targetDir,
    -            String filename, String targetFilename) throws IOException {
    +    public static void forceCreateSymlink(String dir, String targetDir,
    --- End diff --
    
    Deleting it.  I added it then made a better version in fsops, so I am removing it.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77417265
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/SupervisorUtils.java ---
    @@ -159,7 +162,7 @@ public static void addBlobReferences(Localizer localizer, String stormId, Map co
             }
         }
     
    -    public static Set<String> readDownLoadedStormIds(Map conf) throws IOException {
    +    public static Set<String> readDownLoadedStormIds(Map<String, Object> conf) throws IOException {
             Set<String> stormIds = new HashSet<>();
             String path = ConfigUtils.supervisorStormDistRoot(conf);
             Collection<String> rets = Utils.readDirContents(path);
    --- End diff --
    
    I agree that it could be improved. Let's create a separate jira issue to track it. I think it is out-of-scope here.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    Check the ref log, maybe you can recover https://git-scm.com/docs/git-reflog


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

[GitHub] storm pull request #1642: DO NOT MERGE: Please review STORM-2018: Supervisor...

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r76270530
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,494 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.collect.Lists;
    +
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected volatile boolean _exitedEarly = false;
    +    
    +    private class ProcessExitCallback implements Utils.ExitCodeCallable<Void> {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public Void call() throws Exception {
    +            return null;
    +        }
    +
    +        @Override
    +        public Void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +            return null;
    +        }
    +    }
    +    
    +    public BasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, LocalState localState, 
    +            ResourceIsolationInterface resourceIsolationManager, boolean recover) throws IOException {
    +        super(port, assignment, conf, supervisorId, resourceIsolationManager);
    +        _localState = localState;
    +
    +        if (recover) {
    +            synchronized(localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry: workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port +" "+ assignment);
    +                }
    +                _workerId = wid;
    +            }
    +        } else {
    +            createNewWorkerId();
    +        }
    +        
    +        String stormHome = System.getProperty("storm.home");
    +        _profileCmd = stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR + conf.get(Config.WORKER_PROFILER_COMMAND);
    +    }
    +    
    +    public BasicContainer(String workerId, Map<String, Object> conf, String supervisorId,
    +            ResourceIsolationInterface resourceIsolationManager) {
    +        super(-1, null, conf, supervisorId, resourceIsolationManager);
    +        _localState = null;
    +        _workerId = workerId;
    +        _profileCmd = null;
    +    }
    +
    +    protected void createNewWorkerId() {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot create a worker id for a container recovered with just a worker id");
    +        }
    +        synchronized(_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUp() throws IOException {
    +        cleanUpForRestart();
    +        synchronized(_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(_workerId);
    +            _localState.setApprovedWorkers(workersToPort);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        createNewWorkerId();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +    
    +    /**
    +     * Run the given command for profiling
    +     * @param command the command to run
    +     * @param env the environment to run the command
    +     * @param logPrefix the prefix to include in the logs
    +     * @param targetDir the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException on any error
    +     * @throws InterruptedException if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix, File targetDir) throws IOException, InterruptedException {
    +        Process p = Utils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +    
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        if (_port <= 0) {
    +            throw new IllegalStateException("Cannot profile a container recovered with just a worker id");
    +        }
    +        String topologyId = _assignment.get_topology_id();
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, topologyId, _port);
    +        Map<String, Object> topologyConf = ConfigUtils.readSupervisorStormConf(_conf, topologyId);
    +        
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) topologyConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, topologyId, _port);
    +
    +        String workerPid = null;
    +        try (FileReader reader = new FileReader(str); BufferedReader br = new BufferedReader(reader)) {
    +            workerPid = br.readLine().trim();
    +        }
    +        
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        return runProfilingCommand(command, env, logPrefix, targetFile);
    +    }
    +    
    +    private List<String> mkProfileCommand(ProfileAction action, boolean stop, String workerPid, String targetDir) {
    +        if (action == ProfileAction.JMAP_DUMP) {
    +            return jmapDumpCmd(workerPid, targetDir);
    +        } else if (action == ProfileAction.JSTACK_DUMP) {
    +            return jstackDumpCmd(workerPid, targetDir);
    +        } else if (action == ProfileAction.JPROFILE_DUMP) {
    +            return jprofileDump(workerPid, targetDir);
    +        } else if (action == ProfileAction.JVM_RESTART) {
    +            return jprofileJvmRestart(workerPid);
    +        } else if (!stop && action == ProfileAction.JPROFILE_STOP) {
    +            return jprofileStart(workerPid);
    +        } else if (stop && action == ProfileAction.JPROFILE_STOP) {
    +            return jprofileStop(workerPid, targetDir);
    +        }
    +        return Lists.newArrayList();
    +    }
    +
    +    private List<String> jmapDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jmap", targetDir);
    +    }
    +
    +    private List<String> jstackDumpCmd(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "jstack", targetDir);
    +    }
    +
    +    private List<String> jprofileStart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "start");
    +    }
    +
    +    private List<String> jprofileStop(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "stop", targetDir);
    +    }
    +
    +    private List<String> jprofileDump(String pid, String targetDir) {
    +        return Lists.newArrayList(_profileCmd, pid, "dump", targetDir);
    +    }
    +
    +    private List<String> jprofileJvmRestart(String pid) {
    +        return Lists.newArrayList(_profileCmd, pid, "kill");
    +    }
    +    
    +    protected String jlp(String stormRoot, Map<String, Object> conf) {
    +        String resourceRoot = stormRoot + Utils.FILE_PATH_SEPARATOR + ConfigUtils.RESOURCES_SUBDIR;
    +        String os = System.getProperty("os.name").replaceAll("\\s+", "_");
    +        String arch = System.getProperty("os.arch");
    +        String archResourceRoot = resourceRoot + Utils.FILE_PATH_SEPARATOR + os + "-" + arch;
    +        String ret = archResourceRoot + Utils.CLASS_PATH_SEPARATOR + resourceRoot + Utils.CLASS_PATH_SEPARATOR + conf.get(Config.JAVA_LIBRARY_PATH);
    +        return ret;
    +    }
    +    
    +    @SuppressWarnings("unchecked")
    +    protected String getWorkerClassPath(String stormJar, Map<String, Object> topologyConf) {
    +        List<String> topoClasspath = new ArrayList<>();
    +        Object object = topologyConf.get(Config.TOPOLOGY_CLASSPATH);
    +
    +        // Will be populated only if STORM_TOPOLOGY_CLASSPATH_BEGINNING_ENABLED is set on Nimbus.
    +        // Allowed for extreme debugging.
    +        Object topologyClasspathFirst = topologyConf.get(Config.TOPOLOGY_CLASSPATH_BEGINNING);
    +        List<String> firstClasspathList = new ArrayList<>();
    +        if(topologyClasspathFirst instanceof List) {
    +            firstClasspathList.addAll((List<String>)topologyClasspathFirst);
    +        } else if (topologyClasspathFirst instanceof String) {
    +            firstClasspathList.add((String) topologyClasspathFirst);
    +        }
    +        LOG.debug("Topology Classpath Prefix: {}", firstClasspathList);
    +
    +        if (object instanceof List) {
    +            topoClasspath.addAll((List<String>) object);
    +        } else if (object instanceof String) {
    +            topoClasspath.add((String) object);
    +        }
    +        LOG.debug("Topology specific classpath is {}", object);
    +
    +        String classPath = Utils.addToClasspath(firstClasspathList, Arrays.asList(Utils.workerClasspath()));
    +        String classAddPath = Utils.addToClasspath(classPath, Arrays.asList(stormJar));
    +        return Utils.addToClasspath(classAddPath, topoClasspath);
    +    }
    +    
    +    private static String substituteChildOptsInternal(String string,  String workerId, String stormId, int port, int memOnheap) {
    +        if (StringUtils.isNotBlank(string)){
    +            string = string.replace("%ID%", String.valueOf(port));
    +            string = string.replace("%WORKER-ID%", workerId);
    +            string = string.replace("%TOPOLOGY-ID%", stormId);
    +            string = string.replace("%WORKER-PORT%", String.valueOf(port));
    +            string = string.replace("%HEAP-MEM%", String.valueOf(memOnheap));
    +        }
    +        return string;
    +    }
    +    
    +    protected List<String> substituteChildopts(Object value, String workerId, String stormId, int port, int memOnheap) {
    +        List<String> rets = new ArrayList<>();
    +        if (value instanceof String) {
    +            String string = substituteChildOptsInternal((String) value,  workerId, stormId, port, memOnheap);
    +            if (StringUtils.isNotBlank(string)){
    +                String[] strings = string.split("\\s+");
    +                rets.addAll(Arrays.asList(strings));
    +            }
    +        } else if (value instanceof List) {
    +            @SuppressWarnings("unchecked")
    --- End diff --
    
    should we move this to the method itself 


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79034054
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected final String _stormHome = System.getProperty("storm.home");
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +    
    +    /**
    +     * Create a new BasicContainer
    +     * @param type the type of container being made.
    +     * @param conf the supervisor config
    +     * @param supervisorId the ID of the supervisor this is a part of.
    +     * @param port the port the container is on.  Should be <= 0 if only a partial recovery
    +     * @param assignment the assignment for this container. Should be null if only a partial recovery.
    +     * @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used.
    +     * @param localState the local state of the supervisor.  May be null if partial recovery
    +     * @param workerId the id of the worker to use.  Must not be null if doing a partial recovery.
    +     * @param ops file system operations (mostly for testing) if null a new one is made
    +     * @param topoConf the config of the topology (mostly for testing) if null 
    +     * and not a partial recovery the real conf is read.
    +     * @param profileCmd the command to use when profiling (used for testing)
    +     */
    +    public BasicContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int port,
    +            LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
    +            LocalState localState, String workerId, Map<String, Object> topoConf, 
    +            AdvancedFSOps ops, String profileCmd) throws IOException {
    +        super(type, conf, supervisorId, port, assignment, resourceIsolationManager, workerId, topoConf, ops);
    +        assert(localState != null);
    +        _localState = localState;
    +
    +        if (type.isRecovery() && !type.isOnlyKillable()) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    --- End diff --
    
    Great :)


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r78960818
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -0,0 +1,328 @@
    +/**
    + * 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.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 java.util.Map.Entry;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.cluster.VersionedData;
    +import org.apache.storm.daemon.supervisor.Slot.MachineState;
    +import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.ExecutorInfo;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.WorkerResources;
    +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.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ReadClusterState implements Runnable, AutoCloseable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReadClusterState.class);
    +    
    +    private final Map<String, Object> superConf;
    +    private final IStormClusterState stormClusterState;
    +    private final EventManager syncSupEventManager;
    +    private final AtomicReference<Map<String, VersionedData<Assignment>>> assignmentVersions;
    +    private final Map<Integer, Slot> slots = new HashMap<>();
    +    private final AtomicInteger readRetry = new AtomicInteger(0);
    +    private final String assignmentId;
    +    private final ISupervisor iSuper;
    +    private final ILocalizer localizer;
    +    private final ContainerLauncher launcher;
    +    private final String host;
    +    private final LocalState localState;
    +    private final IStormClusterState clusterState;
    +    private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
    +    
    +    public ReadClusterState(Supervisor supervisor) throws Exception {
    +        this.superConf = supervisor.getConf();
    +        this.stormClusterState = supervisor.getStormClusterState();
    +        this.syncSupEventManager = supervisor.getEventManger();
    +        this.assignmentVersions = new AtomicReference<>(new HashMap<>());
    +        this.assignmentId = supervisor.getAssignmentId();
    +        this.iSuper = supervisor.getiSupervisor();
    +        this.localizer = supervisor.getAsyncLocalizer();
    +        this.host = supervisor.getHostName();
    +        this.localState = supervisor.getLocalState();
    +        this.clusterState = supervisor.getStormClusterState();
    +        this.cachedAssignments = supervisor.getCurrAssignment();
    +        
    +        this.launcher = ContainerLauncher.make(superConf, assignmentId, supervisor.getSharedContext());
    +        
    +        @SuppressWarnings("unchecked")
    +        List<Number> ports = (List<Number>)superConf.get(Config.SUPERVISOR_SLOTS_PORTS);
    +        for (Number port: ports) {
    +            slots.put(port.intValue(), mkSlot(port.intValue()));
    +        }
    +        
    +        try {
    +            Collection<String> workers = SupervisorUtils.supervisorWorkerIds(superConf);
    +            for (Slot slot: slots.values()) {
    +                String workerId = slot.getWorkerId();
    +                if (workerId != null) {
    +                    workers.remove(workerId);
    +                }
    +            }
    +            if (!workers.isEmpty()) {
    +                supervisor.killWorkers(workers, launcher);
    +            }
    +        } catch (Exception e) {
    +            LOG.warn("Error trying to clean up old workers", e);
    +        }
    +
    +        //All the slots/assignments should be recovered now, so we can clean up anything that we don't expect to be here
    +        try {
    +            localizer.cleanupUnusedTopologies();
    +        } catch (Exception e) {
    +            LOG.warn("Error trying to clean up old topologies", e);
    +        }
    +        
    +        for (Slot slot: slots.values()) {
    +            slot.start();
    +        }
    +    }
    +
    +    private Slot mkSlot(int port) throws Exception {
    +        return new Slot(localizer, superConf, launcher, host, port,
    +                localState, clusterState, iSuper, cachedAssignments);
    +    }
    +    
    +    @Override
    +    public synchronized void run() {
    +        try {
    +            Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager);
    +            List<String> stormIds = stormClusterState.assignments(syncCallback);
    +            Map<String, VersionedData<Assignment>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback);
    +            
    +            Map<Integer, LocalAssignment> allAssignments =
    +                    readAssignments(assignmentsSnapshot, assignmentId, readRetry);
    +            if (allAssignments == null) {
    +                //Something odd happened try again later
    +                return;
    +            }
    +            Map<String, List<ProfileRequest>> topoIdToProfilerActions = getProfileActions(stormClusterState, stormIds);
    +            
    +            HashSet<Integer> assignedPorts = new HashSet<>();
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("All assignment: {}", allAssignments);
    +            LOG.debug("Topology Ids -> Profiler Actions {}", topoIdToProfilerActions);
    +            for (Integer port: allAssignments.keySet()) {
    +                if (iSuper.confirmAssigned(port)) {
    +                    assignedPorts.add(port);
    +                }
    +            }
    +            HashSet<Integer> allPorts = new HashSet<>(assignedPorts);
    +            allPorts.addAll(slots.keySet());
    +            
    +            Map<Integer, Set<TopoProfileAction>> filtered = new HashMap<>();
    +            for (Entry<String, List<ProfileRequest>> entry: topoIdToProfilerActions.entrySet()) {
    +                String topoId = entry.getKey();
    +                if (entry.getValue() != null) {
    +                    for (ProfileRequest req: entry.getValue()) {
    +                        NodeInfo ni = req.get_nodeInfo();
    +                        if (host.equals(ni.get_node())) {
    +                            Long port = ni.get_port().iterator().next();
    +                            Set<TopoProfileAction> actions = filtered.get(port);
    --- End diff --
    
    Nitpick: There's a computeIfAbsent method on Map that seems ideal for this kind of thing.


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

[GitHub] storm issue #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/1642
  
    Not sure what has been happening with travis not being able to get to the apache maven repo all the time, but my build in travis passed https://travis-ci.org/revans2/incubator-storm/builds/157725399 despite the official build failing.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79026674
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/ContainerLauncher.java ---
    @@ -0,0 +1,103 @@
    +/**
    + * 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.Map;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.messaging.IContext;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Launches containers
    + */
    +public abstract class ContainerLauncher {
    +    private static final Logger LOG = LoggerFactory.getLogger(ContainerLauncher.class);
    +    
    +    /**
    +     * Factory to create the right container launcher 
    +     * for the config and the environment.
    +     * @param conf the config
    +     * @param supervisorId the ID of the supervisor
    +     * @param sharedContext Used in local mode to let workers talk together without netty
    +     * @return the proper container launcher
    +     * @throws IOException on any error
    +     */
    +    public static ContainerLauncher make(Map<String, Object> conf, String supervisorId, IContext sharedContext) throws IOException {
    +        if (ConfigUtils.isLocalMode(conf)) {
    +            return new LocalContainerLauncher(conf, supervisorId, sharedContext);
    +        }
    +        
    +        ResourceIsolationInterface resourceIsolationManager = null;
    +        if (Utils.getBoolean(conf.get(Config.STORM_RESOURCE_ISOLATION_PLUGIN_ENABLE), false)) {
    +            resourceIsolationManager = Utils.newInstance((String) conf.get(Config.STORM_RESOURCE_ISOLATION_PLUGIN));
    +            resourceIsolationManager.prepare(conf);
    +            LOG.info("Using resource isolation plugin {} {}", conf.get(Config.STORM_RESOURCE_ISOLATION_PLUGIN), resourceIsolationManager);
    +        }
    +
    +        if (Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
    +            return new RunAsUserContainerLauncher(conf, supervisorId, resourceIsolationManager);
    +        }
    +        return new BasicContainerLauncher(conf, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    protected ContainerLauncher() {
    +        //Empty
    +    }
    +
    +    /**
    +     * Launch a container in a given slot
    +     * @param port the port to run this on
    +     * @param assignment what to launch
    +     * @param state the current state of the supervisor
    +     * @return The container that can be used to manager the processes.
    +     * @throws IOException on any error 
    +     */
    +    public abstract Container launchContainer(int port, LocalAssignment assignment, LocalState state) throws IOException;
    +    
    +    /**
    +     * Recover a container for a running process
    +     * @param port the port the assignment is running on
    +     * @param assignment the assignment that was launched
    +     * @param state the current state of the supervisor
    +     * @return The container that can be used to manage the processes.
    +     * @throws IOException on any error
    +     */
    +    public abstract Container recoverContainer(int port, LocalAssignment assignment, LocalState state) throws IOException;
    --- End diff --
    
    Yes I have been meaning to switch it over to throw an exception instead of returning null.  Will do that


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77299756
  
    --- Diff: storm-core/test/jvm/org/apache/storm/daemon/supervisor/BasicContainerTest.java ---
    @@ -0,0 +1,459 @@
    +package org.apache.storm.daemon.supervisor;
    +
    +import static org.junit.Assert.*;
    +import static org.mockito.Mockito.*;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.junit.Test;
    +
    +public class BasicContainerTest {
    +    public static class CommandRun {
    +        final List<String> cmd;
    +        final Map<String, String> env;
    +        final File pwd;
    +        
    +        public CommandRun(List<String> cmd, Map<String, String> env, File pwd) {
    +            this.cmd = cmd;
    +            this.env = env;
    +            this.pwd = pwd;
    +        }
    +    }
    +    
    +    public static class MockBasicContainer extends BasicContainer {
    +        public final List<CommandRun> profileCmds = new ArrayList<>();
    +        public final List<CommandRun> workerCmds = new ArrayList<>();
    +        
    +        public MockBasicContainer(int port, LocalAssignment assignment, Map<String, Object> conf,
    +                String supervisorId, LocalState localState, ResourceIsolationInterface resourceIsolationManager,
    +                boolean recover) throws IOException {
    +            super(port, assignment, conf, supervisorId, localState, resourceIsolationManager, recover);
    +        }
    +        
    +        public MockBasicContainer(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +                Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +                ResourceIsolationInterface resourceIsolationManager, LocalState localState,
    +                String profileCmd) throws IOException {
    +            super(ops, port, assignment, conf, topoConf, supervisorId, resourceIsolationManager, localState, profileCmd);
    +        }
    +        
    +        @Override
    +        protected Map<String, Object> readTopoConf() throws IOException {
    +            return new HashMap<>();
    +        }
    +        
    +        @Override
    +        public void createNewWorkerId() {
    +            super.createNewWorkerId();
    +        }
    +        
    +        @Override
    +        public List<String> substituteChildopts(Object value, int memOnheap) {
    +            return super.substituteChildopts(value, memOnheap);
    +        }
    +               
    +        @Override
    +        protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +                File targetDir) throws IOException, InterruptedException {
    +            profileCmds.add(new CommandRun(command, env, targetDir));
    +            return true;
    +        }
    +        
    +        @Override
    +        protected void launchWorkerProcess(List<String> command, Map<String, String> env, String logPrefix,
    +                ExitCodeCallback processExitCallback, File targetDir) throws IOException {
    +            workerCmds.add(new CommandRun(command, env, targetDir));
    +        }
    +        
    +        @Override
    +        protected String javaCmd(String cmd) {
    +            //avoid system dependent things
    +            return cmd;
    +        }
    +        
    +        @Override
    +        protected List<String> frameworkClasspath() {
    +            //We are not really running anything so make this
    +            // simple to check for
    +            return Arrays.asList("FRAMEWORK_CP");
    +        }
    +        
    +        @Override
    +        protected String javaLibraryPath(String stormRoot, Map<String, Object> conf) {
    +            return "JLP";
    +        }
    +    }
    +    
    +    @Test
    +    public void testCreateNewWorkerId() throws Exception {
    +        final String topoId = "test_topology";
    +        final int port = 8080;
    +        LocalAssignment la = new LocalAssignment();
    +        la.set_topology_id(topoId);
    +        
    +        AdvancedFSOps ops = mock(AdvancedFSOps.class);
    +        
    +        LocalState ls = mock(LocalState.class);
    +        
    +        MockBasicContainer mc = new MockBasicContainer(ops, port, la, new HashMap<String, Object>(), 
    +                new HashMap<String, Object>(), "SUPERVISOR", null, ls, "profile");
    +        
    +        mc.createNewWorkerId();
    +        
    +        assertNotNull(mc._workerId);
    +        verify(ls).getApprovedWorkers();
    +        Map<String, Integer> expectedNewState = new HashMap<String, Integer>();
    +        expectedNewState.put(mc._workerId, port);
    +        verify(ls).setApprovedWorkers(expectedNewState);
    +    }
    +    
    +    @Test
    +    public void testRecovery() throws Exception {
    +        final String topoId = "test_topology";
    +        final String workerId = "myWorker";
    +        final int port = 8080;
    +        LocalAssignment la = new LocalAssignment();
    +        la.set_topology_id(topoId);
    +        
    +        Map<String, Integer> workerState = new HashMap<String, Integer>();
    +        workerState.put(workerId, port);
    +        
    +        LocalState ls = mock(LocalState.class);
    +        when(ls.getApprovedWorkers()).thenReturn(workerState);
    +        
    +        MockBasicContainer mc = new MockBasicContainer(port, la, new HashMap<String, Object>(), 
    +                "SUPERVISOR", ls, null, true);
    +        
    +        assertEquals(workerId, mc._workerId);
    +    }
    +    
    +    @Test
    +    public void testRecoveryMiss() throws Exception {
    +        final String topoId = "test_topology";
    +        final int port = 8080;
    +        LocalAssignment la = new LocalAssignment();
    +        la.set_topology_id(topoId);
    +        
    +        Map<String, Integer> workerState = new HashMap<String, Integer>();
    +        workerState.put("somethingelse", port+1);
    +        
    +        LocalState ls = mock(LocalState.class);
    +        when(ls.getApprovedWorkers()).thenReturn(workerState);
    +        
    +        try {
    +            new MockBasicContainer(port, la, new HashMap<String, Object>(), 
    +                    "SUPERVISOR", ls, null, true);
    +            fail("Container recovered worker incorrectly");
    +        } catch (ContainerRecoveryException e) {
    +            //Expected
    +        }
    +    }
    +    
    +    @Test
    +    public void testCleanUp() throws Exception {
    +        final String topoId = "test_topology";
    +        final int port = 8080;
    +        final String workerId = "worker-id";
    +        LocalAssignment la = new LocalAssignment();
    +        la.set_topology_id(topoId);
    +        
    +        AdvancedFSOps ops = mock(AdvancedFSOps.class);
    +        
    +        Map<String, Integer> workerState = new HashMap<String, Integer>();
    +        workerState.put(workerId, port);
    +        
    +        LocalState ls = mock(LocalState.class);
    +        when(ls.getApprovedWorkers()).thenReturn(new HashMap<>(workerState));
    +        
    +        MockBasicContainer mc = new MockBasicContainer(ops, port, la, new HashMap<String, Object>(), 
    +                new HashMap<String, Object>(), "SUPERVISOR", null, ls, "profile");
    +        mc._workerId = workerId;
    +        
    +        mc.cleanUp();
    +        
    +        assertNull(mc._workerId);
    +        verify(ls).getApprovedWorkers();
    +        Map<String, Integer> expectedNewState = new HashMap<String, Integer>();
    +        verify(ls).setApprovedWorkers(expectedNewState);
    +    }
    +    
    +    @Test
    +    public void testRunProfiling() throws Exception {
    +        final long pid = 100;
    +        final String topoId = "test_topology";
    +        final int port = 8080;
    +        final String workerId = "worker-id";
    +        final String stormLocal = ContainerTest.asAbsPath("tmp", "testing");
    +        final String topoRoot = ContainerTest.asAbsPath(stormLocal, topoId, String.valueOf(port));
    +        final File workerArtifactsPid = ContainerTest.asAbsFile(topoRoot, "worker.pid");
    +        
    +        final Map<String, Object> superConf = new HashMap<>();
    +        superConf.put(Config.STORM_LOCAL_DIR, stormLocal);
    +        superConf.put(Config.STORM_WORKERS_ARTIFACTS_DIR, stormLocal);
    +        
    +        LocalAssignment la = new LocalAssignment();
    +        la.set_topology_id(topoId);
    +        
    +        AdvancedFSOps ops = mock(AdvancedFSOps.class);
    +        when(ops.slurpString(workerArtifactsPid)).thenReturn(String.valueOf(pid));
    +        
    +        LocalState ls = mock(LocalState.class);
    +        
    +        MockBasicContainer mc = new MockBasicContainer(ops, port, la, superConf, 
    +                new HashMap<String, Object>(), "SUPERVISOR", null, ls, "profile");
    +        mc._workerId = workerId;
    +        
    +        //HEAP DUMP
    +        ProfileRequest req = new ProfileRequest();
    +        req.set_action(ProfileAction.JMAP_DUMP);
    +        
    +        mc.runProfiling(req, false);
    +        
    +        assertEquals(1, mc.profileCmds.size());
    +        CommandRun cmd = mc.profileCmds.get(0);
    +        mc.profileCmds.clear();
    +        assertEquals(Arrays.asList("profile", String.valueOf(pid), "jmap", topoRoot), cmd.cmd);
    +        assertEquals(new File(topoRoot), cmd.pwd);
    +        
    +        //JSTACK DUMP
    +        req.set_action(ProfileAction.JSTACK_DUMP);
    +        
    +        mc.runProfiling(req, false);
    +        
    +        assertEquals(1, mc.profileCmds.size());
    +        cmd = mc.profileCmds.get(0);
    +        mc.profileCmds.clear();
    +        assertEquals(Arrays.asList("profile", String.valueOf(pid), "jstack", topoRoot), cmd.cmd);
    +        assertEquals(new File(topoRoot), cmd.pwd);
    +        
    +        //RESTART
    +        req.set_action(ProfileAction.JVM_RESTART);
    +        
    +        mc.runProfiling(req, false);
    +        
    +        assertEquals(1, mc.profileCmds.size());
    +        cmd = mc.profileCmds.get(0);
    +        mc.profileCmds.clear();
    +        assertEquals(Arrays.asList("profile", String.valueOf(pid), "kill"), cmd.cmd);
    +        assertEquals(new File(topoRoot), cmd.pwd);
    +        
    +        //JPROFILE DUMP
    +        req.set_action(ProfileAction.JPROFILE_DUMP);
    +        
    +        mc.runProfiling(req, false);
    +        
    +        assertEquals(1, mc.profileCmds.size());
    +        cmd = mc.profileCmds.get(0);
    +        mc.profileCmds.clear();
    +        assertEquals(Arrays.asList("profile", String.valueOf(pid), "dump", topoRoot), cmd.cmd);
    +        assertEquals(new File(topoRoot), cmd.pwd);
    +        
    +        //JPROFILE START
    +        req.set_action(ProfileAction.JPROFILE_STOP);
    --- End diff --
    
    Never mind. I found that mkProfileCommand() only handles JPROFILE_STOP and determine whether start or stop by flag.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79038947
  
    --- 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);
    +        
    --- End diff --
    
    Does that matter if it's only evaluated when running tests?


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79027965
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected final String _stormHome = System.getProperty("storm.home");
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +    
    +    /**
    +     * Create a new BasicContainer
    +     * @param type the type of container being made.
    +     * @param conf the supervisor config
    +     * @param supervisorId the ID of the supervisor this is a part of.
    +     * @param port the port the container is on.  Should be <= 0 if only a partial recovery
    +     * @param assignment the assignment for this container. Should be null if only a partial recovery.
    +     * @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used.
    +     * @param localState the local state of the supervisor.  May be null if partial recovery
    +     * @param workerId the id of the worker to use.  Must not be null if doing a partial recovery.
    +     * @param ops file system operations (mostly for testing) if null a new one is made
    +     * @param topoConf the config of the topology (mostly for testing) if null 
    +     * and not a partial recovery the real conf is read.
    +     * @param profileCmd the command to use when profiling (used for testing)
    +     */
    +    public BasicContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int port,
    --- End diff --
    
    Yes, but the subclasses need access to them too for testing...  I guess they are all in the same package at this point so I'll see how it goes.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79034020
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/BasicContainer.java ---
    @@ -0,0 +1,644 @@
    +/**
    + * 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.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileAction;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.generated.WorkerResources;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * A container that runs processes on the local box.
    + */
    +public class BasicContainer extends Container {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    private static final FilenameFilter jarFilter = new FilenameFilter() {
    +        @Override
    +        public boolean accept(File dir, String name) {
    +            return name.endsWith(".jar");
    +        }
    +    };
    +    private static final Joiner CPJ = 
    +            Joiner.on(Utils.CLASS_PATH_SEPARATOR).skipNulls();
    +    
    +    protected final LocalState _localState;
    +    protected final String _profileCmd;
    +    protected final String _stormHome = System.getProperty("storm.home");
    +    protected volatile boolean _exitedEarly = false;
    +
    +    private class ProcessExitCallback implements ExitCodeCallback {
    +        private final String _logPrefix;
    +
    +        public ProcessExitCallback(String logPrefix) {
    +            _logPrefix = logPrefix;
    +        }
    +
    +        @Override
    +        public void call(int exitCode) {
    +            LOG.info("{} exited with code: {}", _logPrefix, exitCode);
    +            _exitedEarly = true;
    +        }
    +    }
    +    
    +    /**
    +     * Create a new BasicContainer
    +     * @param type the type of container being made.
    +     * @param conf the supervisor config
    +     * @param supervisorId the ID of the supervisor this is a part of.
    +     * @param port the port the container is on.  Should be <= 0 if only a partial recovery
    +     * @param assignment the assignment for this container. Should be null if only a partial recovery.
    +     * @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used.
    +     * @param localState the local state of the supervisor.  May be null if partial recovery
    +     * @param workerId the id of the worker to use.  Must not be null if doing a partial recovery.
    +     * @param ops file system operations (mostly for testing) if null a new one is made
    +     * @param topoConf the config of the topology (mostly for testing) if null 
    +     * and not a partial recovery the real conf is read.
    +     * @param profileCmd the command to use when profiling (used for testing)
    +     */
    +    public BasicContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int port,
    +            LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
    +            LocalState localState, String workerId, Map<String, Object> topoConf, 
    +            AdvancedFSOps ops, String profileCmd) throws IOException {
    +        super(type, conf, supervisorId, port, assignment, resourceIsolationManager, workerId, topoConf, ops);
    +        assert(localState != null);
    +        _localState = localState;
    +
    +        if (type.isRecovery() && !type.isOnlyKillable()) {
    +            synchronized (localState) {
    +                String wid = null;
    +                Map<String, Integer> workerToPort = localState.getApprovedWorkers();
    +                for (Map.Entry<String, Integer> entry : workerToPort.entrySet()) {
    +                    if (port == entry.getValue().intValue()) {
    +                        wid = entry.getKey();
    +                    }
    +                }
    +                if (wid == null) {
    +                    throw new ContainerRecoveryException("Could not find worker id for " + port + " " + assignment);
    +                }
    +                LOG.info("Recovered Worker {}", wid);
    +                _workerId = wid;
    +            }
    +        } else if (_workerId == null){
    +            createNewWorkerId();
    +        }
    +
    +        if (profileCmd == null) {
    +            profileCmd = _stormHome + Utils.FILE_PATH_SEPARATOR + "bin" + Utils.FILE_PATH_SEPARATOR
    +                    + conf.get(Config.WORKER_PROFILER_COMMAND);
    +        }
    +        _profileCmd = profileCmd;
    +    }
    +
    +    /**
    +     * Create a new worker ID for this process and store in in this object and
    +     * in the local state.  Never call this if a worker is currently up and running.
    +     * We will lose track of the process.
    +     */
    +    protected void createNewWorkerId() {
    +        _type.assertFull();
    +        assert(_workerId == null);
    +        synchronized (_localState) {
    +            _workerId = Utils.uuid();
    +            Map<String, Integer> workerToPort = _localState.getApprovedWorkers();
    +            if (workerToPort == null) {
    +                workerToPort = new HashMap<>(1);
    +            }
    +            removeWorkersOn(workerToPort, _port);
    +            workerToPort.put(_workerId, _port);
    +            _localState.setApprovedWorkers(workerToPort);
    +            LOG.info("Created Worker ID {}", _workerId);
    +        }
    +    }
    +
    +    private static void removeWorkersOn(Map<String, Integer> workerToPort, int _port) {
    +        for (Iterator<Entry<String, Integer>> i = workerToPort.entrySet().iterator(); i.hasNext();) {
    +            Entry<String, Integer> found = i.next();
    +            if (_port == found.getValue().intValue()) {
    +                LOG.warn("Deleting worker {} from state", found.getKey());
    +                i.remove();
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void cleanUpForRestart() throws IOException {
    +        String origWorkerId = _workerId;
    +        super.cleanUpForRestart();
    +        synchronized (_localState) {
    +            Map<String, Integer> workersToPort = _localState.getApprovedWorkers();
    +            workersToPort.remove(origWorkerId);
    +            removeWorkersOn(workersToPort, _port);
    +            _localState.setApprovedWorkers(workersToPort);
    +            LOG.info("Removed Worker ID {}", origWorkerId);
    +        }
    +    }
    +
    +    @Override
    +    public void relaunch() throws IOException {
    +        _type.assertFull();
    +        //We are launching it now...
    +        _type = ContainerType.LAUNCH;
    +        createNewWorkerId();
    +        setup();
    +        launch();
    +    }
    +
    +    @Override
    +    public boolean didMainProcessExit() {
    +        return _exitedEarly;
    +    }
    +
    +    /**
    +     * Run the given command for profiling
    +     * 
    +     * @param command
    +     *            the command to run
    +     * @param env
    +     *            the environment to run the command
    +     * @param logPrefix
    +     *            the prefix to include in the logs
    +     * @param targetDir
    +     *            the working directory to run the command in
    +     * @return true if it ran successfully, else false
    +     * @throws IOException
    +     *             on any error
    +     * @throws InterruptedException
    +     *             if interrupted wile waiting for the process to exit.
    +     */
    +    protected boolean runProfilingCommand(List<String> command, Map<String, String> env, String logPrefix,
    +            File targetDir) throws IOException, InterruptedException {
    +        _type.assertFull();
    +        Process p = SupervisorUtils.launchProcess(command, env, logPrefix, null, targetDir);
    +        int ret = p.waitFor();
    +        return ret == 0;
    +    }
    +
    +    @Override
    +    public boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException {
    +        _type.assertFull();
    +        String targetDir = ConfigUtils.workerArtifactsRoot(_conf, _topologyId, _port);
    +
    +        @SuppressWarnings("unchecked")
    +        Map<String, String> env = (Map<String, String>) _topoConf.get(Config.TOPOLOGY_ENVIRONMENT);
    +        if (env == null) {
    +            env = new HashMap<String, String>();
    +        }
    +
    +        String str = ConfigUtils.workerArtifactsPidPath(_conf, _topologyId, _port);
    +
    +        String workerPid = _ops.slurpString(new File(str)).trim();
    +
    +        ProfileAction profileAction = request.get_action();
    +        String logPrefix = "ProfilerAction process " + _topologyId + ":" + _port + " PROFILER_ACTION: " + profileAction
    +                + " ";
    +
    +        List<String> command = mkProfileCommand(profileAction, stop, workerPid, targetDir);
    +
    +        File targetFile = new File(targetDir);
    +        if (command.size() > 0) {
    +            return runProfilingCommand(command, env, logPrefix, targetFile);
    +        }
    +        LOG.warn("PROFILING REQUEST NOT SUPPORTED {} IGNORED...", request);
    +        return true;
    --- End diff --
    
    Oh right, that makes sense. I wasn't looking at the place this is used, just went by the Javadoc for the overridden method.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r79033063
  
    --- 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);
    +        
    +        //Ignore changes to scheduling while downloading the topology blobs
    +        // We don't support canceling the download through the future yet,
    +        // so to keep everything in sync, just wait
    +        try {
    +            dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS);
    +            //Downloading of all blobs finished.
    +            if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) {
    +                //Scheduling changed
    +                staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization, staticState.port);
    +                return prepareForNewAssignmentOnEmptySlot(dynamicState, staticState);
    +            }
    +            Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
    +            return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null);
    +        } catch (TimeoutException e) {
    +            //We waited for 1 second loop around and try again....
    +            return dynamicState;
    +        }
    +    }
    +    
    +    /**
    +     * State Transitions for WAITING_FOR_BASIC_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 handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception {
    +        assert(dynamicState.pendingLocalization != null);
    +        assert(dynamicState.pendingDownload != null);
    +        assert(dynamicState.container == null);
    --- End diff --
    
    Again the only way this is called is if the state is WaitingForBasicLocalization.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77419710
  
    --- Diff: storm-core/src/jvm/org/apache/storm/utils/ConfigUtils.java ---
    @@ -353,25 +350,21 @@ public LocalState nimbusTopoHistoryStateImpl(Map conf) throws IOException {
         }
     
         // we use this "weird" wrapper pattern temporarily for mocking in clojure test
    -    public static Map readSupervisorStormConf(Map conf, String stormId) throws IOException {
    +    public static Map<String, Object> readSupervisorStormConf(Map<String, Object> conf, String stormId) throws IOException {
             return _instance.readSupervisorStormConfImpl(conf, stormId);
         }
     
    -    public Map readSupervisorStormConfImpl(Map conf, String stormId) throws IOException {
    +    public Map<String, Object> readSupervisorStormConfImpl(Map<String, Object> conf, String stormId) throws IOException {
             String stormRoot = supervisorStormDistRoot(conf, stormId);
             String confPath = supervisorStormConfPath(stormRoot);
             return readSupervisorStormConfGivenPath(conf, confPath);
         }
     
         // we use this "weird" wrapper pattern temporarily for mocking in clojure test
    -    public static StormTopology readSupervisorTopology(Map conf, String stormId) throws IOException {
    -        return _instance.readSupervisorTopologyImpl(conf, stormId);
    -    }
    -
    -    public StormTopology readSupervisorTopologyImpl(Map conf, String stormId) throws IOException {
    +    public static StormTopology readSupervisorTopology(Map conf, String stormId, AdvancedFSOps ops) throws IOException {
    --- End diff --
    
    Agreed. The comment describes a pattern that this change removes.


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1642#discussion_r77442272
  
    --- Diff: storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java ---
    @@ -0,0 +1,493 @@
    +/**
    + * 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.BufferedReader;
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.io.Reader;
    +import java.io.Writer;
    +import java.lang.ProcessBuilder.Redirect;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.ProfileRequest;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.LocalState;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +/**
    + * Represents a container that a worker will run in.
    + */
    +public abstract class Container implements Killable {
    +    private static final Logger LOG = LoggerFactory.getLogger(BasicContainer.class);
    +    protected final Map<String, Object> _conf;
    +    protected final Map<String, Object> _topoConf;
    +    protected String _workerId;
    +    protected final String _topologyId;
    +    protected final String _supervisorId;
    +    protected final int _port;
    +    protected final LocalAssignment _assignment;
    +    protected final AdvancedFSOps _ops;
    +    protected final ResourceIsolationInterface _resourceIsolationManager;
    +    
    +    //Exposed for testing
    +    protected Container(AdvancedFSOps ops, int port, LocalAssignment assignment,
    +            Map<String, Object> conf, Map<String, Object> topoConf, String supervisorId, 
    +            ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        assert((assignment == null && port <= 0) ||
    +                (assignment != null && port > 0));
    +        assert(conf != null);
    +        assert(ops != null);
    +        assert(supervisorId != null);
    +        
    +        _port = port;
    +        _ops = ops;
    +        _assignment = assignment;
    +        if (assignment != null) {
    +            _topologyId = assignment.get_topology_id();
    +        } else {
    +            _topologyId = null;
    +        }
    +        _conf = conf;
    +        _supervisorId = supervisorId;
    +        _resourceIsolationManager = resourceIsolationManager;
    +        if (topoConf == null) {
    +            _topoConf = readTopoConf();
    +        } else {
    +            _topoConf = topoConf;
    +        }
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return this.getClass().getSimpleName() + " topo:" + _topologyId + " worker:" + _workerId;
    +    }
    +    
    +    protected Map<String, Object> readTopoConf() throws IOException {
    +        assert(_topologyId != null);
    +        return ConfigUtils.readSupervisorStormConf(_conf, _topologyId);
    +    }
    +    
    +    protected Container(int port, LocalAssignment assignment, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), port, assignment, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Constructor to use when trying to recover a container from just the worker ID.
    +     * @param workerId the id of the worker
    +     * @param conf the config of the supervisor
    +     * @param supervisorId the id of the supervisor
    +     * @param resourceIsolationManager the isolation manager.
    +     * @throws IOException on any error
    +     */
    +    protected Container(String workerId, Map<String, Object> conf, 
    +            String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException {
    +        this(AdvancedFSOps.make(conf), -1, null, conf, null, supervisorId, resourceIsolationManager);
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void kill(long pid) throws IOException {
    +        Utils.killProcessWithSigTerm(String.valueOf(pid));
    +    }
    +    
    +    /**
    +     * Kill a given process
    +     * @param pid the id of the process to kill
    +     * @throws IOException
    +     */
    +    protected void forceKill(long pid) throws IOException {
    +        Utils.forceKillProcess(String.valueOf(pid));
    +    }
    +    
    +    @Override
    +    public void kill() throws IOException {
    +        LOG.info("Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +
    +        for (Long pid : pids) {
    +            kill(pid);
    +        }
    +    }
    +    
    +    @Override
    +    public void forceKill() throws IOException {
    +        LOG.info("Force Killing {}:{}", _supervisorId, _workerId);
    +        Set<Long> pids = getAllPids();
    +        
    +        for (Long pid : pids) {
    +            forceKill(pid);
    +        }
    +    }
    +    
    +    /**
    +     * Read the Heartbeat for the current container.
    +     * @return the Heartbeat
    +     * @throws IOException on any error
    +     */
    +    public LSWorkerHeartbeat readHeartbeat() throws IOException {
    +        LocalState localState = ConfigUtils.workerState(_conf, _workerId);
    +        LSWorkerHeartbeat hb = localState.getWorkerHeartBeat();
    +        LOG.trace("{}: Reading heartbeat {}", _workerId, hb);
    +        return hb;
    +    }
    +
    +    /**
    +     * Is a process alive and running?
    +     * @param pid the PID of the running process
    +     * @param user the user that is expected to own that process
    +     * @return true if it is, else false
    +     * @throws IOException on any error
    +     */
    +    protected boolean isProcessAlive(long pid, String user) throws IOException {
    +        if (Utils.IS_ON_WINDOWS) {
    +            return isWindowsProcessAlive(pid, user);
    +        }
    +        return isPosixProcessAlive(pid, user);
    +    }
    +    
    +    private boolean isWindowsProcessAlive(long pid, String user) throws IOException {
    +        boolean ret = false;
    --- End diff --
    
    I took at stab at it https://github.com/revans2/incubator-storm/pull/6


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

[GitHub] storm pull request #1642: STORM-2018: Supervisor V2.

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 closed the pull request at:

    https://github.com/apache/storm/pull/1642


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