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

Reply via email to